http://git-wip-us.apache.org/repos/asf/kylin/blob/2425bf0c/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 544e072..12446ff 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 @@ -201,7 +201,7 @@ public class SparkCubingByLayer extends AbstractApplication implements Serializa if (initialized == false) { prepare(); long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc); - Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId); + Cuboid baseCuboid = Cuboid.findById(cubeSegment, baseCuboidId); baseCuboidBuilder = new BaseCuboidBuilder(kylinConfig, cubeDesc, cubeSegment, intermediateTableDesc, AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid), MeasureIngester.create(cubeDesc.getMeasures()), cubeSegment.buildDictionaryMap()); initialized = true; } @@ -244,7 +244,7 @@ public class SparkCubingByLayer extends AbstractApplication implements Serializa reducerFunction2 = new CuboidReducerFunction2(measureNum, vCubeDesc.getValue(), measureAggregators, needAggr); } - final int totalLevels = cubeDesc.getBuildLevel(); + final int totalLevels = cubeSegment.getCuboidScheduler().getBuildLevel(); JavaPairRDD<ByteArray, Object[]>[] allRDDs = new JavaPairRDD[totalLevels + 1]; int level = 0; int partition = estimateRDDPartitionNum(level, cubeStatsReader, kylinConfig); @@ -360,9 +360,9 @@ public class SparkCubingByLayer extends AbstractApplication implements Serializa byte[] key = tuple2._1().array(); long cuboidId = rowKeySplitter.split(key); - Cuboid parentCuboid = Cuboid.findById(cubeDesc, cuboidId); + Cuboid parentCuboid = Cuboid.findById(cubeSegment, cuboidId); - Collection<Long> myChildren = cubeDesc.getCuboidScheduler().getSpanningCuboid(cuboidId); + Collection<Long> myChildren = cubeSegment.getCuboidScheduler().getSpanningCuboid(cuboidId); // if still empty or null if (myChildren == null || myChildren.size() == 0) { @@ -371,7 +371,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(cubeDesc, child); + Cuboid childCuboid = Cuboid.findById(cubeSegment, child); Pair<Integer, ByteArray> result = ndCuboidBuilder.buildKey(parentCuboid, childCuboid, rowKeySplitter.getSplitBuffers()); byte[] newKey = new byte[result.getFirst()];
http://git-wip-us.apache.org/repos/asf/kylin/blob/2425bf0c/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java ---------------------------------------------------------------------- diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java index 793cd87..d7b20c8 100644 --- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java +++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java @@ -65,7 +65,7 @@ public final class DefaultTupleConverter implements TupleConverter { @Override public final Tuple2<byte[], byte[]> convert(long cuboidId, GTRecord record) { - Cuboid cuboid = Cuboid.findById(segment.getCubeDesc(), cuboidId); + Cuboid cuboid = Cuboid.findById(segment, cuboidId); RowKeyEncoder rowkeyEncoder = rowKeyEncoderProvider.getRowkeyEncoder(cuboid); final int dimensions = Long.bitCount(cuboidId); http://git-wip-us.apache.org/repos/asf/kylin/blob/2425bf0c/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java ---------------------------------------------------------------------- diff --git a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java index a19b8c7..5e5b16a 100644 --- a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java +++ b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderStressTest.java @@ -84,7 +84,7 @@ public class ITDoggedCubeBuilderStressTest extends LocalFileMetadataTestCase { long randSeed = System.currentTimeMillis(); IJoinedFlatTableDesc flatDesc = EngineFactory.getJoinedFlatTableDesc(cube.getDescriptor()); - DoggedCubeBuilder doggedBuilder = new DoggedCubeBuilder(cube.getDescriptor(), flatDesc, dictionaryMap); + DoggedCubeBuilder doggedBuilder = new DoggedCubeBuilder(cube.getCuboidScheduler(), flatDesc, dictionaryMap); doggedBuilder.setConcurrentThreads(THREADS); { http://git-wip-us.apache.org/repos/asf/kylin/blob/2425bf0c/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java ---------------------------------------------------------------------- diff --git a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java index dbd9ce2..1e10d79 100644 --- a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java +++ b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITDoggedCubeBuilderTest.java @@ -88,7 +88,7 @@ public class ITDoggedCubeBuilderTest extends LocalFileMetadataTestCase { long randSeed = System.currentTimeMillis(); IJoinedFlatTableDesc flatDesc = EngineFactory.getJoinedFlatTableDesc(cube.getDescriptor()); - DoggedCubeBuilder doggedBuilder = new DoggedCubeBuilder(cube.getDescriptor(), flatDesc, dictionaryMap); + DoggedCubeBuilder doggedBuilder = new DoggedCubeBuilder(cube.getCuboidScheduler(), flatDesc, dictionaryMap); doggedBuilder.setConcurrentThreads(THREADS); doggedBuilder.setSplitRowThreshold(SPLIT_ROWS); FileRecordWriter doggedResult = new FileRecordWriter(); @@ -100,7 +100,7 @@ public class ITDoggedCubeBuilderTest extends LocalFileMetadataTestCase { doggedResult.close(); } - InMemCubeBuilder inmemBuilder = new InMemCubeBuilder(cube.getDescriptor(), flatDesc, dictionaryMap); + InMemCubeBuilder inmemBuilder = new InMemCubeBuilder(cube.getCuboidScheduler(), flatDesc, dictionaryMap); inmemBuilder.setConcurrentThreads(THREADS); FileRecordWriter inmemResult = new FileRecordWriter(); http://git-wip-us.apache.org/repos/asf/kylin/blob/2425bf0c/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java ---------------------------------------------------------------------- diff --git a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java index ea66c73..49d267a 100644 --- a/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java +++ b/kylin-it/src/test/java/org/apache/kylin/cube/inmemcubing/ITInMemCubeBuilderTest.java @@ -109,7 +109,7 @@ public class ITInMemCubeBuilderTest extends LocalFileMetadataTestCase { private void testBuildInner() throws Exception { IJoinedFlatTableDesc flatDesc = EngineFactory.getJoinedFlatTableDesc(cube.getDescriptor()); - InMemCubeBuilder cubeBuilder = new InMemCubeBuilder(cube.getDescriptor(), flatDesc, dictionaryMap); + InMemCubeBuilder cubeBuilder = new InMemCubeBuilder(cube.getCuboidScheduler(), flatDesc, dictionaryMap); //DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap); cubeBuilder.setConcurrentThreads(nThreads); http://git-wip-us.apache.org/repos/asf/kylin/blob/2425bf0c/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 6587d4e..59f1349 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,7 @@ 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(cubeDesc, cuboidId)); + rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, Cuboid.findById(cubeSegment, cuboidId)); keybuf = rowKeyEncoder.createBuf(); } rowKeyEncoder.encode(record, record.getInfo().getPrimaryKey(), keybuf); @@ -92,7 +92,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(cubeDesc, cuboidId); + final Cuboid cuboid = Cuboid.findById(cubeSegment, cuboidId); final int nDims = cuboid.getColumns().size(); final ImmutableBitSet bitSet = new ImmutableBitSet(nDims, nDims + cubeDesc.getMeasures().size()); http://git-wip-us.apache.org/repos/asf/kylin/blob/2425bf0c/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/MeasureTypeOnlyAggrInBaseTest.java ---------------------------------------------------------------------- diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/MeasureTypeOnlyAggrInBaseTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/MeasureTypeOnlyAggrInBaseTest.java index f151876..7d089ac 100644 --- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/MeasureTypeOnlyAggrInBaseTest.java +++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/MeasureTypeOnlyAggrInBaseTest.java @@ -91,10 +91,10 @@ public class MeasureTypeOnlyAggrInBaseTest extends LocalFileMetadataTestCase { @Test public void testIdentifyCuboidV2() throws InvocationTargetException, NoSuchMethodException, IllegalAccessException, NoSuchFieldException { CubeDesc cubeDesc = cube.getDescriptor(); - Cuboid ret = Cuboid.identifyCuboid(cubeDesc, Sets.<TblColRef> newHashSet(), Lists.<FunctionDesc> newArrayList()); + Cuboid ret = Cuboid.identifyCuboid(cube, Sets.<TblColRef> newHashSet(), Lists.<FunctionDesc> newArrayList()); long baseCuboidId = cubeDesc.getRowkey().getFullMask(); assertNotEquals(baseCuboidId, ret.getId()); - ret = Cuboid.identifyCuboid(cubeDesc, dimensions, metrics); + ret = Cuboid.identifyCuboid(cube, dimensions, metrics); assertEquals(baseCuboidId, ret.getId()); }
