Repository: incubator-kylin Updated Branches: refs/heads/2.x-staging c3880be69 -> 1923cd429
KYLIN-980 check available memory and then spill Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/1923cd42 Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/1923cd42 Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/1923cd42 Branch: refs/heads/2.x-staging Commit: 1923cd429470498b58c67ce39e7160bb658b8a97 Parents: c3880be Author: shaofengshi <shaofeng...@apache.org> Authored: Tue Nov 24 17:35:23 2015 +0800 Committer: shaofengshi <shaofeng...@apache.org> Committed: Tue Nov 24 17:35:23 2015 +0800 ---------------------------------------------------------------------- .../engine/mr/steps/FactDistinctColumnsReducer.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/1923cd42/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java ---------------------------------------------------------------------- diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java index 66d63fb..ddd366c 100644 --- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java +++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java @@ -19,26 +19,22 @@ package org.apache.kylin.engine.mr.steps; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.*; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; import org.apache.kylin.common.KylinConfig; import org.apache.kylin.common.hll.HyperLogLogPlusCounter; import org.apache.kylin.common.util.ByteArray; import org.apache.kylin.common.util.Bytes; +import org.apache.kylin.common.util.MemoryBudgetController; import org.apache.kylin.cube.CubeInstance; import org.apache.kylin.cube.CubeManager; -import org.apache.kylin.cube.cuboid.Cuboid; -import org.apache.kylin.cube.kv.RowConstants; import org.apache.kylin.cube.model.CubeDesc; import org.apache.kylin.engine.mr.KylinReducer; import org.apache.kylin.engine.mr.common.AbstractHadoopJob; @@ -92,14 +88,12 @@ public class FactDistinctColumnsReducer extends KylinReducer<LongWritable, Text, TblColRef col = columnList.get((int) key.get()); HashSet<ByteArray> set = new HashSet<ByteArray>(); - - Text textValue = null; - Iterator<Text> valueItr = values.iterator(); - while (valueItr.hasNext()) { - textValue = valueItr.next(); + int count = 0; + for (Text textValue : values) { ByteArray value = new ByteArray(Bytes.copy(textValue.getBytes(), 0, textValue.getLength())); set.add(value); - if (set.size() >= 5000000) { // output when count reach 5 Million + count++; + if (count % 10000 == 0 && MemoryBudgetController.getSystemAvailMB() < 100) { outputDistinctValues(col, set, context); set.clear(); }