PageRenderTime 58ms CodeModel.GetById 25ms RepoModel.GetById 1ms app.codeStats 0ms

/sensei-core/src/main/java/com/senseidb/search/req/mapred/functions/groupby/GroupByMapReduceJob.java

https://bitbucket.org/icosplays/sensei
Java | 417 lines | 355 code | 43 blank | 19 comment | 88 complexity | cdcad2ac498497609473aa1e57ac06ac MD5 | raw file
  1. package com.senseidb.search.req.mapred.functions.groupby;
  2. import it.unimi.dsi.fastutil.longs.Long2ObjectArrayMap;
  3. import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
  4. import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
  5. import java.io.Serializable;
  6. import java.util.ArrayList;
  7. import java.util.Collections;
  8. import java.util.HashMap;
  9. import java.util.Iterator;
  10. import java.util.List;
  11. import java.util.Map;
  12. import java.util.PriorityQueue;
  13. import org.json.JSONArray;
  14. import org.json.JSONException;
  15. import org.json.JSONObject;
  16. import scala.actors.threadpool.Arrays;
  17. import com.alibaba.fastjson.util.IdentityHashMap;
  18. import com.browseengine.bobo.api.BoboIndexReader;
  19. import com.browseengine.bobo.facets.data.TermValueList;
  20. import com.browseengine.bobo.util.BigSegmentedArray;
  21. import com.senseidb.search.req.mapred.CombinerStage;
  22. import com.senseidb.search.req.mapred.FacetCountAccessor;
  23. import com.senseidb.search.req.mapred.FieldAccessor;
  24. import com.senseidb.search.req.mapred.IntArray;
  25. import com.senseidb.search.req.mapred.SenseiMapReduce;
  26. import com.senseidb.search.req.mapred.SingleFieldAccessor;
  27. import com.senseidb.util.JSONUtil;
  28. class MapResult implements Serializable {
  29. @SuppressWarnings("rawtypes")
  30. public MapResult(int initialCapacity, TermValueList[] dictionaries, BoboIndexReader indexReader) {
  31. this.dictionaries = dictionaries;
  32. this.indexReader = indexReader;
  33. results = new Long2ObjectOpenHashMap<GroupedValue>(initialCapacity);
  34. }
  35. public Long2ObjectOpenHashMap<GroupedValue> results;
  36. public TermValueList[] dictionaries;
  37. public BoboIndexReader indexReader;
  38. @Override
  39. public String toString() {
  40. return "MapResult [results=" + results + ", dictionaries=" + java.util.Arrays.toString(dictionaries) + "]";
  41. }
  42. }
  43. public class GroupByMapReduceJob implements SenseiMapReduce<Serializable, HashMap<String, GroupedValue>> {
  44. public static final int TRIM_SIZE = 200;
  45. private String[] columns;
  46. private String metric;
  47. private String function;
  48. private AggregateFunction aggregateFunction;
  49. private int top = 10;
  50. @Override
  51. public void init(JSONObject params) {
  52. try {
  53. metric = params.getString("metric");
  54. function = params.getString("function");
  55. aggregateFunction = AggregateFunctionFactory.valueOf(function, metric);
  56. JSONArray columnsJson = params.getJSONArray("columns");
  57. columns = new String[columnsJson.length()];
  58. top = params.optInt("top", 10);
  59. for (int i = 0; i < columnsJson.length(); i++) {
  60. columns[i] = columnsJson.getString(i);
  61. }
  62. } catch (JSONException ex) {
  63. throw new RuntimeException(ex);
  64. }
  65. }
  66. @Override
  67. public Serializable map(IntArray docIds, int docIdCount, long[] uids, FieldAccessor accessor, FacetCountAccessor facetCountsAccessor) {
  68. SingleFieldAccessor singleFieldAccessor = "count".equalsIgnoreCase(function) ? null : accessor.getSingleFieldAccessor(metric);
  69. TermValueList[] dictionaries = new TermValueList[columns.length];
  70. for (int i = 0; i < columns.length; i++) {
  71. dictionaries[i] = accessor.getTermValueList(columns[i]);
  72. }
  73. MapResult mapResult = new MapResult(TRIM_SIZE, dictionaries, accessor.getBoboIndexReader());
  74. SingleFieldAccessor[] orders = new SingleFieldAccessor[columns.length];
  75. for (int i = 0; i < columns.length; i++) {
  76. orders[i] = accessor.getSingleFieldAccessor(columns[i]);
  77. }
  78. int[] numBits = new int[columns.length];
  79. int totalBitSet = 0;
  80. for (int i = 0; i < columns.length; i++) {
  81. numBits[i] = BitHacks.findLogBase2(dictionaries[i].size()) + 1;
  82. totalBitSet += numBits[i];
  83. }
  84. if (totalBitSet > 64) {
  85. throw new IllegalArgumentException("Too many columns for an efficient group by");
  86. }
  87. for (int i = 0; i < docIdCount; i++) {
  88. long key = getKey(dictionaries, orders, numBits, docIds.get(i));
  89. GroupedValue value = mapResult.results.get(key);
  90. if (value != null) {
  91. value.merge(aggregateFunction.produceSingleValue(singleFieldAccessor, docIds.get(i)));
  92. } else {
  93. mapResult.results.put(key, aggregateFunction.produceSingleValue(singleFieldAccessor, docIds.get(i)));
  94. }
  95. }
  96. if (mapResult.results.size() > Math.max(TRIM_SIZE, top) * 20) {
  97. trimToSize(mapResult.results, Math.max(TRIM_SIZE, top) * 5);
  98. }
  99. return mapResult;
  100. }
  101. private long getKey(TermValueList[] dictionaries, SingleFieldAccessor[] orders, int[] numBits, int docId) {
  102. long ret = 0L;
  103. int i = 0;
  104. //StringBuilder b = new StringBuilder();
  105. while (true) {
  106. if (i >= numBits.length) {
  107. break;
  108. }
  109. ret = ret << numBits[i];
  110. ret |= orders[i].getDictionaryId(docId);
  111. // b.append(dictionaries[i].get(orders[i].get(docId)));
  112. if (i >= numBits.length - 1) {
  113. break;
  114. }
  115. i++;
  116. }
  117. //System.out.println(b.toString() + "\n" + decodeKey(new String[dictionaries.length], dictionaries, numBits, ret));
  118. return ret;
  119. }
  120. private String decodeKey(String[] str, TermValueList[] dictionaries, int[] numBits, long key) {
  121. int i = numBits.length - 1;
  122. while (i >= 0) {
  123. long number = key & (-1L >>> (64 - numBits[i]));
  124. str[i] = dictionaries[i].get((int) number);
  125. key >>>= numBits[i];
  126. i--;
  127. }
  128. StringBuilder builder = new StringBuilder();
  129. for (int j = 0; j < str.length - 1; j++) {
  130. builder.append(str[j]).append(":");
  131. }
  132. builder.append(str[str.length - 1]);
  133. return builder.toString();
  134. }
  135. @Override
  136. public List<Serializable> combine(List<Serializable> mapResults, CombinerStage combinerStage) {
  137. // System.out.println("Combine - " + mapResults);
  138. /*
  139. * if (combinerStage == CombinerStage.partitionLevel) { if (map == null)
  140. * { return Collections.EMPTY_LIST; } trimToSize(map, TRIM_SIZE * 5);
  141. * List<HashMap<String, GroupedValue>> ret =
  142. * java.util.Arrays.asList(map); map = new HashMap<String,
  143. * GroupedValue>(); return ret; }
  144. */
  145. if (combinerStage == CombinerStage.partitionLevel) {
  146. if (mapResults.size() == 0) {
  147. return Collections.EMPTY_LIST;
  148. }
  149. if (mapResults.size() == 1) {
  150. HashMap<String, GroupedValue> ret = convert((MapResult) mapResults.get(0));
  151. return java.util.Arrays.asList((Serializable)ret);
  152. }
  153. HashMap<BoboIndexReader, MapResult> results = new HashMap<BoboIndexReader, MapResult>();
  154. for (int i = 0; i < mapResults.size(); i++) {
  155. MapResult current = (MapResult) mapResults.get(i);
  156. if (results.get(current.indexReader) != null) {
  157. Long2ObjectOpenHashMap<GroupedValue> currentMergedResults = results.get(current.indexReader).results;
  158. Long2ObjectOpenHashMap<GroupedValue> currentResultsToMerge = current.results;
  159. for (long key : currentResultsToMerge.keySet()) {
  160. GroupedValue groupedValue = currentMergedResults.get(key);
  161. if (groupedValue != null) {
  162. groupedValue.merge(currentResultsToMerge.get(key));
  163. } else {
  164. currentMergedResults.put(key, currentResultsToMerge.get(key));
  165. }
  166. }
  167. // .putAll(currentResultsToMerge);
  168. trimToSize(currentResultsToMerge, Math.max(TRIM_SIZE, top));
  169. } else {
  170. results.put(current.indexReader, current);
  171. }
  172. }
  173. HashMap<String, GroupedValue> ret = null;
  174. for (BoboIndexReader key : results.keySet()) {
  175. if (ret == null ) {
  176. ret = convert(results.get(key));
  177. } else {
  178. merge(ret, convert(results.get(key)));
  179. }
  180. }
  181. // System.out.println("End combine - " + ret);
  182. return java.util.Arrays.asList((Serializable)ret);
  183. }
  184. if (mapResults.size() == 0) {
  185. return Collections.EMPTY_LIST;
  186. }
  187. if (mapResults.size() == 1) {
  188. HashMap<String, GroupedValue> ret = (HashMap<String, GroupedValue>) mapResults.get(0);
  189. return java.util.Arrays.asList((Serializable)ret);
  190. }
  191. HashMap<String, GroupedValue> firstMap = (HashMap<String, GroupedValue>) mapResults.get(0);
  192. for (int i = 1; i < mapResults.size(); i++) {
  193. merge(firstMap, (HashMap<String, GroupedValue>) mapResults.get(i));
  194. }
  195. trimToSize(firstMap, Math.max(TRIM_SIZE, top));
  196. return java.util.Arrays.asList((Serializable)firstMap);
  197. }
  198. private HashMap<String, GroupedValue> convert(MapResult mapResult) {
  199. HashMap<String, GroupedValue> ret = new HashMap<String, GroupedValue>(mapResult.results.size());
  200. String[] temp = new String[mapResult.dictionaries.length];
  201. int[] numBits = new int[columns.length];
  202. for (int i = 0; i < columns.length; i++) {
  203. numBits[i] = BitHacks.findLogBase2(mapResult.dictionaries[i].size()) + 1;
  204. }
  205. for (long key : mapResult.results.keySet()) {
  206. ret.put(decodeKey(temp, mapResult.dictionaries, numBits, key), mapResult.results.get(key));
  207. }
  208. return ret;
  209. }
  210. private void merge(HashMap<String, GroupedValue> firstMap, HashMap<String, GroupedValue> secondMap) {
  211. for (Map.Entry<String, GroupedValue> entry : secondMap.entrySet()) {
  212. GroupedValue groupedValue = firstMap.get(entry.getKey());
  213. if (groupedValue != null) {
  214. groupedValue.merge(entry.getValue());
  215. } else {
  216. firstMap.put(entry.getKey(), entry.getValue());
  217. }
  218. }
  219. }
  220. @Override
  221. public HashMap<String, GroupedValue> reduce(List<Serializable> combineResultsRaw) {
  222. List<HashMap<String, GroupedValue>> combineResults = (List) combineResultsRaw;
  223. if (combineResults.size() == 0) {
  224. return null;
  225. }
  226. if (combineResults.size() == 1) {
  227. return combineResults.get(0);
  228. }
  229. HashMap<String, GroupedValue> firstMap = combineResults.get(0);
  230. for (int i = 1; i < combineResults.size(); i++) {
  231. merge(firstMap, combineResults.get(i));
  232. }
  233. trimToSize(firstMap, TRIM_SIZE);
  234. return firstMap;
  235. }
  236. /**
  237. * Tries to trim the map to smaller size
  238. *
  239. * @param map
  240. * @param count
  241. */
  242. private static void trimToSize(Map<String, ? extends Comparable> map, int count) {
  243. if (map.size() < count) {
  244. return;
  245. }
  246. double trimRatio = ((double) count) / map.size() * 2;
  247. if (trimRatio >= 1.0D) {
  248. return;
  249. }
  250. int queueSize = (int) (map.size() / Math.log(map.size()) / 4);
  251. PriorityQueue<Comparable> queue = new PriorityQueue<Comparable>(queueSize);
  252. int i = 0;
  253. int addElementRange = map.size() / queueSize;
  254. for (Comparable groupedValue : map.values()) {
  255. if (i == addElementRange) {
  256. i = 0;
  257. queue.add(groupedValue);
  258. } else {
  259. i++;
  260. }
  261. }
  262. int elementIndex = (int) (queue.size() * (1.0d - trimRatio));
  263. if (elementIndex >= queue.size()) {
  264. elementIndex = queue.size() - 1;
  265. }
  266. int counter = 0;
  267. Comparable newMinimumValue = null;
  268. while (!queue.isEmpty()) {
  269. if (counter == elementIndex) {
  270. newMinimumValue = queue.poll();
  271. break;
  272. } else {
  273. counter++;
  274. queue.poll();
  275. }
  276. }
  277. if (newMinimumValue == null) {
  278. return;
  279. }
  280. Iterator<? extends Comparable> iterator = map.values().iterator();
  281. int numToRemove = map.size() - count;
  282. counter = 0;
  283. while (iterator.hasNext()) {
  284. if (iterator.next().compareTo(newMinimumValue) <= 0) {
  285. counter++;
  286. iterator.remove();
  287. if (counter >= numToRemove) {
  288. break;
  289. }
  290. }
  291. }
  292. }
  293. private static void trimToSize(Long2ObjectMap<? extends Comparable> map, int count) {
  294. if (map.size() < count) {
  295. return;
  296. }
  297. double trimRatio = ((double) count) / map.size() * 2;
  298. if (trimRatio >= 1.0D) {
  299. return;
  300. }
  301. int queueSize = (int) (map.size() / Math.log(map.size()) / 4);
  302. PriorityQueue<Comparable> queue = new PriorityQueue<Comparable>(queueSize);
  303. int i = 0;
  304. int addElementRange = map.size() / queueSize;
  305. for (Comparable groupedValue : map.values()) {
  306. if (i == addElementRange) {
  307. i = 0;
  308. queue.add(groupedValue);
  309. } else {
  310. i++;
  311. }
  312. }
  313. int elementIndex = (int) (queue.size() * (1.0d - trimRatio));
  314. if (elementIndex >= queue.size()) {
  315. elementIndex = queue.size() - 1;
  316. }
  317. int counter = 0;
  318. Comparable newMinimumValue = null;
  319. while (!queue.isEmpty()) {
  320. if (counter == elementIndex) {
  321. newMinimumValue = queue.poll();
  322. break;
  323. } else {
  324. counter++;
  325. queue.poll();
  326. }
  327. }
  328. if (newMinimumValue == null) {
  329. return;
  330. }
  331. Iterator<? extends Comparable> iterator = map.values().iterator();
  332. int numToRemove = map.size() - count;
  333. counter = 0;
  334. while (iterator.hasNext()) {
  335. if (iterator.next().compareTo(newMinimumValue) <= 0) {
  336. counter++;
  337. iterator.remove();
  338. if (counter >= numToRemove) {
  339. break;
  340. }
  341. }
  342. }
  343. }
  344. @Override
  345. public JSONObject render(HashMap<String, GroupedValue> reduceResult) {
  346. try {
  347. Object result = aggregateFunction.toJson(reduceResult);
  348. if (result instanceof JSONObject) {
  349. return (JSONObject) result;
  350. } else if (result instanceof JSONArray) {
  351. JSONArray jsonArrResult = (JSONArray) result;
  352. if (jsonArrResult.length() > top) {
  353. JSONArray newArr = new JSONUtil.FastJSONArray();
  354. for (int i = 0; i < top; i++) {
  355. newArr.put(jsonArrResult.get(i));
  356. }
  357. jsonArrResult = newArr;
  358. }
  359. return new JSONUtil.FastJSONObject().put("grouped", jsonArrResult).put("column", metric);
  360. } else {
  361. return new JSONUtil.FastJSONObject().put("grouped", result).put("column", metric);
  362. }
  363. } catch (JSONException e) {
  364. throw new RuntimeException(e);
  365. }
  366. }
  367. private String getKey(String[] columns, FieldAccessor fieldAccessor, int docId) {
  368. StringBuilder key = new StringBuilder(fieldAccessor.get(columns[0], docId).toString());
  369. for (int i = 1; i < columns.length; i++) {
  370. key.append(":").append(fieldAccessor.get(columns[i], docId).toString());
  371. }
  372. return key.toString();
  373. }
  374. }