Skip to content

Commit 6f9bce2

Browse files
author
gaodayue
committed
KYLIN-2501 pipeline partition results if possible
1 parent 7cb88f5 commit 6f9bce2

File tree

1 file changed

+12
-10
lines changed

1 file changed

+12
-10
lines changed

core-storage/src/main/java/org/apache/kylin/storage/gtrecord/StorageResponseGTScatter.java

+12-10
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818

1919
package org.apache.kylin.storage.gtrecord;
2020

21+
import com.google.common.base.Function;
2122
import com.google.common.collect.Iterators;
2223
import com.google.common.collect.Lists;
2324
import org.apache.kylin.common.util.ImmutableBitSet;
@@ -69,21 +70,22 @@ public void close() throws IOException {
6970

7071
@Override
7172
public Iterator<GTRecord> iterator() {
72-
List<PartitionResultIterator> partitionResults = Lists.newArrayList();
73-
while (blocks.hasNext()) {
74-
partitionResults.add(new PartitionResultIterator(blocks.next(), info, columns));
73+
Iterator<PartitionResultIterator> iterators = Iterators.transform(blocks, new Function<byte[], PartitionResultIterator>() {
74+
public PartitionResultIterator apply(byte[] input) {
75+
return new PartitionResultIterator(input, info, columns);
76+
}
77+
});
78+
79+
if (!needSorted) {
80+
logger.debug("Using Iterators.concat to pipeline partition results");
81+
return Iterators.concat(iterators);
7582
}
7683

84+
List<PartitionResultIterator> partitionResults = Lists.newArrayList(iterators);
7785
if (partitionResults.size() == 1) {
7886
return partitionResults.get(0);
7987
}
80-
81-
if (!needSorted) {
82-
logger.debug("Using Iterators.concat to merge partition results");
83-
return Iterators.concat(partitionResults.iterator());
84-
}
85-
86-
logger.debug("Using SortMergedPartitionResultIterator to merge partition results");
88+
logger.debug("Using SortMergedPartitionResultIterator to merge {} partition results", partitionResults.size());
8789
return new SortMergedPartitionResultIterator(partitionResults, info, GTRecord.getComparator(groupByDims));
8890
}
8991
}

0 commit comments

Comments
 (0)