KYLIN-2493 Auto enlarge buffer in FactDistinctColumnsMapper Signed-off-by: Hongbin Ma <mahong...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/7eb9969e Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/7eb9969e Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/7eb9969e Branch: refs/heads/master-hbase0.98 Commit: 7eb9969e1ce54cc919ad47f2589d85da44090d18 Parents: 182ed00 Author: xiefan46 <958034...@qq.com> Authored: Thu Mar 9 18:25:11 2017 +0800 Committer: Hongbin Ma <mahong...@apache.org> Committed: Thu Mar 9 21:19:30 2017 +0800 ---------------------------------------------------------------------- .../mr/steps/FactDistinctColumnsMapper.java | 24 +++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/7eb9969e/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java ---------------------------------------------------------------------- diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java index d9c1309..07e788c 100644 --- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java +++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java @@ -48,7 +48,7 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB public static enum RawDataCounter { BYTES - }; + } protected boolean collectStatistics = false; protected CuboidScheduler cuboidScheduler = null; @@ -153,8 +153,13 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB } tmpbuf.clear(); + byte[] valueBytes = Bytes.toBytes(fieldValue); + int size = valueBytes.length + 1; + if (size >= tmpbuf.capacity()) { + tmpbuf = ByteBuffer.allocate(countNewSize(tmpbuf.capacity(), size)); + } tmpbuf.put(Bytes.toBytes(reducerIndex)[3]); - tmpbuf.put(Bytes.toBytes(fieldValue)); + tmpbuf.put(valueBytes); outputKey.set(tmpbuf.array(), 0, tmpbuf.position()); sortableKey.setText(outputKey); //judge type @@ -176,8 +181,13 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB String fieldValue = row[partitionColumnIndex]; if (fieldValue != null) { tmpbuf.clear(); + byte[] valueBytes = Bytes.toBytes(fieldValue); + int size = valueBytes.length + 1; + if (size >= tmpbuf.capacity()) { + tmpbuf = ByteBuffer.allocate(countNewSize(tmpbuf.capacity(), size)); + } tmpbuf.put(MARK_FOR_PARTITION_COL); - tmpbuf.put(Bytes.toBytes(fieldValue)); + tmpbuf.put(valueBytes); outputKey.set(tmpbuf.array(), 0, tmpbuf.position()); sortableKey.setText(outputKey); sortableKey.setTypeId((byte) 0); @@ -243,4 +253,12 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB } } } + + private int countNewSize(int oldSize, int dataSize) { + int newSize = oldSize * 2; + while (newSize < dataSize) { + newSize = newSize * 2; + } + return newSize; + } }