APACHE-KYLIN-2733: Change findById to findForMandatory during cube building


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/0f13305d
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/0f13305d
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/0f13305d

Branch: refs/heads/master
Commit: 0f13305dd42705aa0344e61fb843a1412d2b1dc7
Parents: 99fbd75
Author: Zhong <nju_y...@apache.org>
Authored: Tue Nov 7 02:36:16 2017 +0800
Committer: Zhong <nju_y...@apache.org>
Committed: Sat Dec 2 23:21:43 2017 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/cube/cuboid/Cuboid.java     |  9 +--------
 .../java/org/apache/kylin/cube/CubeInstanceTest.java  | 14 +++++++++-----
 .../org/apache/kylin/cube/kv/RowKeyDecoderTest.java   |  2 +-
 .../org/apache/kylin/cube/kv/RowKeyEncoderTest.java   |  6 +++---
 .../kylin/engine/mr/steps/MergeCuboidMapper.java      |  2 +-
 .../apache/kylin/engine/mr/steps/NDCuboidMapper.java  |  2 +-
 .../apache/kylin/engine/spark/SparkCubingByLayer.java |  6 +++---
 .../kylin/storage/hbase/steps/HBaseCuboidWriter.java  |  5 +++--
 8 files changed, 22 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0f13305d/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java 
b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
index 3c4fceb..d9c7803 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
@@ -93,18 +93,11 @@ public class Cuboid implements Comparable<Cuboid>, 
Serializable {
         return new Cuboid(cube, cuboidID, cuboidID);
     }
 
-    public static Cuboid findById(CuboidScheduler cuboidScheduler, byte[] 
cuboidID) {
-        return findById(cuboidScheduler, Bytes.toLong(cuboidID));
-    }
-
+    @Deprecated
     public static Cuboid findById(CubeSegment cubeSegment, long cuboidID) {
         return findById(cubeSegment.getCuboidScheduler(), cuboidID);
     }
 
-    public static Cuboid findById(CubeInstance cubeInstance, long cuboidID) {
-        return findById(cubeInstance.getCuboidScheduler(), cuboidID);
-    }
-
     @VisibleForTesting
     static Cuboid findById(CubeDesc cubeDesc, long cuboidID) {
         return findById(cubeDesc.getInitialCuboidScheduler(), cuboidID);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0f13305d/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java 
b/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java
index 0af5f98..9e517b9 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeInstanceTest.java
@@ -18,9 +18,13 @@
 
 package org.apache.kylin.cube;
 
+import static org.junit.Assert.assertNotNull;
+
 import java.io.DataInputStream;
 import java.io.FileInputStream;
 import java.io.InputStream;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.kylin.common.persistence.JsonSerializer;
@@ -46,12 +50,12 @@ public class CubeInstanceTest {
         Map<Long, Long> cuboidsWithRowCnt = cubeInstance.getCuboids();
         
TreeCuboidScheduler.CuboidTree.createFromCuboids(Lists.newArrayList(cuboidsWithRowCnt.keySet()),
                 new 
TreeCuboidScheduler.CuboidCostComparator(cuboidsWithRowCnt));
-    }
 
-    public void printMap(Map<Long, Long> map) {
-        System.out.println("size: " + map.size());
-        for (Map.Entry<Long, Long> entry : map.entrySet()) {
-            System.out.println(entry.getKey() + ":" + entry.getValue());
+        List<Long> cuboids = Lists.newArrayList(cuboidsWithRowCnt.keySet());
+        Collections.sort(cuboids);
+        for (Long cuboid : cuboids) {
+            System.out.println(cuboid + ":" + cuboidsWithRowCnt.get(cuboid));
         }
+        assertNotNull(cuboidsWithRowCnt.get(255L));
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/0f13305d/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java 
b/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java
index 459e734..d24baf6 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyDecoderTest.java
@@ -91,7 +91,7 @@ public class RowKeyDecoderTest extends 
LocalFileMetadataTestCase {
         data[7] = "15";
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cube, baseCuboidId);
+        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
         RowKeyEncoder rowKeyEncoder = new 
RowKeyEncoder(cube.getFirstSegment(), baseCuboid);
 
         byte[] encodedKey = rowKeyEncoder.encode(data);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0f13305d/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java 
b/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java
index dcd883e..d638d8c 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/kv/RowKeyEncoderTest.java
@@ -66,7 +66,7 @@ public class RowKeyEncoderTest extends 
LocalFileMetadataTestCase {
         data[7] = "15";
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cube, baseCuboidId);
+        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
         RowKeyEncoder rowKeyEncoder = new 
RowKeyEncoder(cube.getFirstSegment(), baseCuboid);
 
         byte[] encodedKey = rowKeyEncoder.encode(data);
@@ -97,7 +97,7 @@ public class RowKeyEncoderTest extends 
LocalFileMetadataTestCase {
         data[8] = "15";
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cube, baseCuboidId);
+        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
         RowKeyEncoder rowKeyEncoder = new 
RowKeyEncoder(cube.getFirstSegment(), baseCuboid);
 
         byte[] encodedKey = rowKeyEncoder.encode(data);
@@ -133,7 +133,7 @@ public class RowKeyEncoderTest extends 
LocalFileMetadataTestCase {
         data[8] = null;
 
         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-        Cuboid baseCuboid = Cuboid.findById(cube, baseCuboidId);
+        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, baseCuboidId);
         RowKeyEncoder rowKeyEncoder = new 
RowKeyEncoder(cube.getFirstSegment(), baseCuboid);
 
         byte[] encodedKey = rowKeyEncoder.encode(data);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0f13305d/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
index 4f68293..2c85a28 100755
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -147,7 +147,7 @@ public class MergeCuboidMapper extends KylinMapper<Text, 
Text, Text, Text> {
     @Override
     public void doMap(Text key, Text value, Context context) throws 
IOException, InterruptedException {
         long cuboidID = rowKeySplitter.split(key.getBytes());
-        Cuboid cuboid = Cuboid.findById(cube, cuboidID);
+        Cuboid cuboid = Cuboid.findForMandatory(cubeDesc, cuboidID);
         RowKeyEncoder rowkeyEncoder = 
rowKeyEncoderProvider.getRowkeyEncoder(cuboid);
 
         SplittedBytes[] splittedByteses = rowKeySplitter.getSplitBuffers();

http://git-wip-us.apache.org/repos/asf/kylin/blob/0f13305d/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
index 359bd4c..f936393 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
@@ -86,7 +86,7 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, 
Text, Text> {
     @Override
     public void doMap(Text key, Text value, Context context) throws 
IOException, InterruptedException {
         long cuboidId = rowKeySplitter.split(key.getBytes());
-        Cuboid parentCuboid = Cuboid.findById(cuboidScheduler, cuboidId);
+        Cuboid parentCuboid = Cuboid.findForMandatory(cubeDesc, cuboidId);
 
         Collection<Long> myChildren = 
cuboidScheduler.getSpanningCuboid(cuboidId);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/0f13305d/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
----------------------------------------------------------------------
diff --git 
a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
 
b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
index bdf0633..8d75070 100644
--- 
a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
+++ 
b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
@@ -290,7 +290,7 @@ public class SparkCubingByLayer extends AbstractApplication 
implements Serializa
                         CubeJoinedFlatTableEnrich interDesc = new 
CubeJoinedFlatTableEnrich(
                                 
EngineFactory.getJoinedFlatTableDesc(cubeSegment), cubeDesc);
                         long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
-                        Cuboid baseCuboid = Cuboid.findById(cubeInstance, 
baseCuboidId);
+                        Cuboid baseCuboid = Cuboid.findForMandatory(cubeDesc, 
baseCuboidId);
                         baseCuboidBuilder = new BaseCuboidBuilder(kConfig, 
cubeDesc, cubeSegment, interDesc,
                                 
AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid),
                                 
MeasureIngester.create(cubeDesc.getMeasures()), 
cubeSegment.buildDictionaryMap());
@@ -423,7 +423,7 @@ public class SparkCubingByLayer extends AbstractApplication 
implements Serializa
 
             byte[] key = tuple2._1().array();
             long cuboidId = rowKeySplitter.split(key);
-            Cuboid parentCuboid = Cuboid.findById(cubeSegment, cuboidId);
+            Cuboid parentCuboid = Cuboid.findForMandatory(cubeDesc, cuboidId);
 
             Collection<Long> myChildren = 
cubeSegment.getCuboidScheduler().getSpanningCuboid(cuboidId);
 
@@ -434,7 +434,7 @@ public class SparkCubingByLayer extends AbstractApplication 
implements Serializa
 
             List<Tuple2<ByteArray, Object[]>> tuples = new 
ArrayList(myChildren.size());
             for (Long child : myChildren) {
-                Cuboid childCuboid = Cuboid.findById(cubeSegment, child);
+                Cuboid childCuboid = Cuboid.findForMandatory(cubeDesc, child);
                 Pair<Integer, ByteArray> result = 
ndCuboidBuilder.buildKey(parentCuboid, childCuboid,
                         rowKeySplitter.getSplitBuffers());
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/0f13305d/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index 59f1349..e04e07d 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -81,7 +81,8 @@ public class HBaseCuboidWriter implements ICuboidWriter {
     //TODO:shardingonstreaming
     private byte[] createKey(Long cuboidId, GTRecord record) {
         if (rowKeyEncoder == null || rowKeyEncoder.getCuboidID() != cuboidId) {
-            rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, 
Cuboid.findById(cubeSegment, cuboidId));
+            rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment,
+                    Cuboid.findForMandatory(cubeDesc, cuboidId));
             keybuf = rowKeyEncoder.createBuf();
         }
         rowKeyEncoder.encode(record, record.getInfo().getPrimaryKey(), keybuf);
@@ -92,7 +93,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
     @Override
     public void write(long cuboidId, GTRecord record) throws IOException {
         byte[] key = createKey(cuboidId, record);
-        final Cuboid cuboid = Cuboid.findById(cubeSegment, cuboidId);
+        final Cuboid cuboid = Cuboid.findForMandatory(cubeDesc, cuboidId);
         final int nDims = cuboid.getColumns().size();
         final ImmutableBitSet bitSet = new ImmutableBitSet(nDims, nDims + 
cubeDesc.getMeasures().size());
 

Reply via email to