Github user hyunsik commented on a diff in the pull request:
https://github.com/apache/tajo/pull/115#discussion_r16338083
--- Diff:
tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
---
@@ -923,20 +960,62 @@ public static void
scheduleScatteredHashShuffleFetches(TaskSchedulerContext sche
LOG.info(subQuery.getId()
+ ", ShuffleType:" + SCATTERED_HASH_SHUFFLE.name()
- + ", DeterminedTaskNum : " + fetches.size());
+ + ", Intermediate Size: " + totalIntermediateSize
+ + ", splitSize: " + splitVolume
+ + ", DeterminedTaskNum: " + fetches.size());
}
- static class IntermediateEntryComparator implements
Comparator<IntermediateEntry> {
+ /**
+ * If a IntermediateEntry is large than splitVolume, List<FetchImpl> has
single element.
+ * @param ebId
+ * @param entries
+ * @param splitVolume
+ * @return
+ */
+ public static List<List<FetchImpl>> splitOrMergeIntermediates(
+ ExecutionBlockId ebId, List<IntermediateEntry> entries, long
splitVolume, long pageSize) {
+ // Each List<FetchImpl> has splitVolume size.
+ List<List<FetchImpl>> fetches = new ArrayList<List<FetchImpl>>();
+
+ Iterator<IntermediateEntry> iter = entries.iterator();
+ if (!iter.hasNext()) {
+ return null;
+ }
+ List<FetchImpl> fetchListForSingleTask = new ArrayList<FetchImpl>();
+ long fetchListVolume = 0;
- @Override
- public int compare(IntermediateEntry o1, IntermediateEntry o2) {
- int cmp = Ints.compare(o1.getPartId(), o2.getPartId());
- if (cmp != 0) {
- return cmp;
+ while (iter.hasNext()) {
+ IntermediateEntry currentInterm = iter.next();
+
+ long firstSplitVolume = splitVolume - fetchListVolume;
+ if (firstSplitVolume < pageSize) {
+ firstSplitVolume = splitVolume;
+ }
+ List<Pair<Long, Long>> splits =
currentInterm.split(firstSplitVolume, splitVolume);
--- End diff --
Could you add some comments what each long of a pair means?
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---