KYLIN-1137 TopN measure need support dictionary merge KYLIN-1137 column name need be upper case in cube desc
KYLIN-1137 fix bug in MergeCuboidMapper KYLIN-1137 add topN support in MergeCuboidFromStorageMapper for MR_V2 fix wrong fast mode message Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/134960c6 Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/134960c6 Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/134960c6 Branch: refs/heads/2.x-staging Commit: 134960c62424c8f4bd9e4841e925f209caf3d8be Parents: d256a7f Author: shaofengshi <shaofeng...@apache.org> Authored: Wed Nov 11 19:58:02 2015 +0800 Committer: honma <ho...@ebay.com> Committed: Mon Nov 16 10:28:25 2015 +0800 ---------------------------------------------------------------------- .../kylin/job/BuildCubeWithEngineTest.java | 10 +-- .../org/apache/kylin/common/topn/Counter.java | 2 +- .../org/apache/kylin/cube/model/CubeDesc.java | 10 +++ .../engine/mr/steps/InMemCuboidMapper.java | 26 ++---- .../mr/steps/MergeCuboidFromStorageMapper.java | 69 ++++++++++++++ .../engine/mr/steps/MergeCuboidMapper.java | 95 ++++++++++++++++++-- .../engine/mr/steps/MergeDictionaryStep.java | 17 ++-- .../cube_desc/test_kylin_cube_topn_desc.json | 2 +- .../test_kylin_cube_topn_left_join_desc.json | 2 +- .../test_kylin_cube_without_slr_desc.json | 2 +- ...t_kylin_cube_without_slr_left_join_desc.json | 2 +- 11 files changed, 193 insertions(+), 44 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java ---------------------------------------------------------------------- diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java index d2a101d..75a6c54 100644 --- a/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java +++ b/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithEngineTest.java @@ -62,7 +62,7 @@ public class BuildCubeWithEngineTest { private CubeManager cubeManager; private DefaultScheduler scheduler; protected ExecutableManager jobService; - private static boolean fastBuildMode = true; + private static boolean fastBuildMode = false; private static final Log logger = LogFactory.getLog(BuildCubeWithEngineTest.class); @@ -87,11 +87,11 @@ public class BuildCubeWithEngineTest { ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath()); String fastModeStr = System.getProperty("fastBuildMode"); - if (fastModeStr != null && fastModeStr.equalsIgnoreCase("false")) { - fastBuildMode = false; - logger.info("Will not use fast build mode"); - } else { + if (fastModeStr != null && fastModeStr.equalsIgnoreCase("true")) { + fastBuildMode = true; logger.info("Will use fast build mode"); + } else { + logger.info("Will not use fast build mode"); } System.setProperty(KylinConfig.KYLIN_CONF, "../examples/test_case_data/sandbox"); http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/core-common/src/main/java/org/apache/kylin/common/topn/Counter.java ---------------------------------------------------------------------- diff --git a/core-common/src/main/java/org/apache/kylin/common/topn/Counter.java b/core-common/src/main/java/org/apache/kylin/common/topn/Counter.java index 2bca4df..31c5ed1 100644 --- a/core-common/src/main/java/org/apache/kylin/common/topn/Counter.java +++ b/core-common/src/main/java/org/apache/kylin/common/topn/Counter.java @@ -50,7 +50,7 @@ public class Counter<T> implements Externalizable { public T getItem() { return item; } - + public double getCount() { return count; } http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java ---------------------------------------------------------------------- diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java index 95eaf6d..2250945 100644 --- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java +++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java @@ -854,4 +854,14 @@ public class CubeDesc extends RootPersistentEntity { public LinkedHashSet<TblColRef> getMeasureDisplayColumns() { return measureDisplayColumns; } + + + public boolean hasMeasureUsingDictionary() { + for (MeasureDesc measureDesc : this.getMeasures()) { + if (measureDesc.getFunction().isTopN()) + return true; + } + + return false; + } } http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java ---------------------------------------------------------------------- diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java index 2bf627b..d724c76 100644 --- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java +++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java @@ -65,26 +65,14 @@ public class InMemCuboidMapper<KEYIN> extends KylinMapper<KEYIN, Object, ByteArr Map<TblColRef, Dictionary<?>> dictionaryMap = Maps.newHashMap(); - for (DimensionDesc dim : cubeDesc.getDimensions()) { - // dictionary - for (TblColRef col : dim.getColumnRefs()) { - if (cubeDesc.getRowkey().isUseDictionary(col)) { - Dictionary<?> dict = cubeSegment.getDictionary(col); - if (dict == null) { - logger.warn("Dictionary for " + col + " was not found."); - } - - dictionaryMap.put(col, cubeSegment.getDictionary(col)); - } - } - } - - for (MeasureDesc measureDesc : cubeDesc.getMeasures()) { - if (measureDesc.getFunction().isTopN()) { - List<TblColRef> colRefs = measureDesc.getFunction().getParameter().getColRefs(); - TblColRef col = colRefs.get(colRefs.size() - 1); - dictionaryMap.put(col, cubeSegment.getDictionary(col)); + // dictionary + for (TblColRef col : cubeDesc.getAllColumnsNeedDictionary()) { + Dictionary<?> dict = cubeSegment.getDictionary(col); + if (dict == null) { + logger.warn("Dictionary for " + col + " was not found."); } + + dictionaryMap.put(col, cubeSegment.getDictionary(col)); } DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cube.getDescriptor(), dictionaryMap); http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java ---------------------------------------------------------------------- diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java index 50f3d4c..286ff02 100644 --- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java +++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidFromStorageMapper.java @@ -21,8 +21,14 @@ package org.apache.kylin.engine.mr.steps; import java.io.IOException; import java.nio.ByteBuffer; import java.util.HashMap; +import java.util.List; +import com.google.common.collect.Lists; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.io.Text; import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.topn.Counter; +import org.apache.kylin.common.topn.TopNCounter; import org.apache.kylin.common.util.ByteArray; import org.apache.kylin.common.util.BytesUtil; import org.apache.kylin.common.util.Pair; @@ -45,6 +51,8 @@ import org.apache.kylin.engine.mr.MRUtil; import org.apache.kylin.engine.mr.common.AbstractHadoopJob; import org.apache.kylin.engine.mr.common.BatchConstants; import org.apache.kylin.metadata.measure.MeasureCodec; +import org.apache.kylin.metadata.model.ColumnDesc; +import org.apache.kylin.metadata.model.MeasureDesc; import org.apache.kylin.metadata.model.SegmentStatusEnum; import org.apache.kylin.metadata.model.TblColRef; import org.slf4j.Logger; @@ -81,6 +89,9 @@ public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, By private MeasureCodec codec; private ByteArrayWritable outputValue = new ByteArrayWritable(); + private List<MeasureDesc> measuresDescs; + private Integer[] measureIdxUsingDict; + private Boolean checkNeedMerging(TblColRef col) throws IOException { Boolean ret = dictsNeedMerging.get(col); if (ret != null) @@ -119,7 +130,18 @@ public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, By rowKeySplitter = new RowKeySplitter(sourceCubeSegment, 65, 255); rowKeyEncoderProvider = new RowKeyEncoderProvider(mergedCubeSegment); + measuresDescs = cubeDesc.getMeasures(); codec = new MeasureCodec(cubeDesc.getMeasures()); + if (cubeDesc.hasMeasureUsingDictionary()) { + List<Integer> measuresUsingDict = Lists.newArrayList(); + for (int i = 0; i < measuresDescs.size(); i++) { + if (measuresDescs.get(i).getFunction().isTopN()) { + // so far only TopN uses dic + measuresUsingDict.add(i); + } + } + measureIdxUsingDict = measuresUsingDict.toArray(new Integer[measuresUsingDict.size()]); + } } @Override @@ -191,6 +213,11 @@ public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, By rowkeyEncoder.encode(new ByteArray(newKeyBodyBuf, 0, bufOffset), newKeyBuf); outputKey.set(newKeyBuf.array(), 0, fullKeySize); + // encode measure if it uses dictionary + if (cubeDesc.hasMeasureUsingDictionary()) { + reEncodeMeasure(value); + } + valueBuf.clear(); codec.encode(value, valueBuf); outputValue.set(valueBuf.array(), 0, valueBuf.position()); @@ -198,4 +225,46 @@ public class MergeCuboidFromStorageMapper extends KylinMapper<Object, Object, By context.write(outputKey, outputValue); } + private void reEncodeMeasure(Object[] measureObjs) throws IOException, InterruptedException { + int bufOffset = 0; + for (int idx : measureIdxUsingDict) { + // only TopN measure uses dic + TopNCounter<ByteArray> topNCounters = (TopNCounter<ByteArray>) measureObjs[idx]; + + MeasureDesc measureDesc = measuresDescs.get(idx); + String displayCol = measureDesc.getFunction().getParameter().getDisplayColumn().toUpperCase(); + if (StringUtils.isNotEmpty(displayCol)) { + ColumnDesc sourceColumn = cubeDesc.getFactTableDesc().findColumnByName(displayCol); + TblColRef colRef = new TblColRef(sourceColumn); + DictionaryManager dictMgr = DictionaryManager.getInstance(config); + Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(colRef)); + Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(colRef)); + + int topNSize = topNCounters.size(); + while (sourceDict.getSizeOfValue() * topNSize > newKeyBodyBuf.length - bufOffset || // + mergedDict.getSizeOfValue() * topNSize > newKeyBodyBuf.length - bufOffset || // + mergedDict.getSizeOfId() * topNSize > newKeyBodyBuf.length - bufOffset) { + byte[] oldBuf = newKeyBodyBuf; + newKeyBodyBuf = new byte[2 * newKeyBodyBuf.length]; + System.arraycopy(oldBuf, 0, newKeyBodyBuf, 0, oldBuf.length); + } + + for (Counter<ByteArray> c : topNCounters) { + int idInSourceDict = BytesUtil.readUnsigned(c.getItem().array(), c.getItem().offset(), c.getItem().length()); + int idInMergedDict; + int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBodyBuf, bufOffset); + if (size < 0) { + idInMergedDict = mergedDict.nullId(); + } else { + idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBodyBuf, bufOffset, size); + } + + BytesUtil.writeUnsigned(idInMergedDict, newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId()); + c.getItem().set(newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId()); + bufOffset += mergedDict.getSizeOfId(); + } + } + } + + } } http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/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 0b68e59..68d1481 100644 --- 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 @@ -18,14 +18,13 @@ package org.apache.kylin.engine.mr.steps; -import java.io.IOException; -import java.util.HashMap; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - +import com.google.common.collect.Lists; +import org.apache.commons.lang.StringUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.lib.input.FileSplit; import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.common.topn.Counter; +import org.apache.kylin.common.topn.TopNCounter; import org.apache.kylin.common.util.ByteArray; import org.apache.kylin.common.util.BytesUtil; import org.apache.kylin.common.util.SplittedBytes; @@ -43,9 +42,19 @@ import org.apache.kylin.dict.DictionaryManager; import org.apache.kylin.engine.mr.KylinMapper; import org.apache.kylin.engine.mr.common.AbstractHadoopJob; import org.apache.kylin.engine.mr.common.BatchConstants; +import org.apache.kylin.metadata.measure.MeasureCodec; +import org.apache.kylin.metadata.model.ColumnDesc; +import org.apache.kylin.metadata.model.MeasureDesc; import org.apache.kylin.metadata.model.SegmentStatusEnum; import org.apache.kylin.metadata.model.TblColRef; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + /** * @author ysong1, honma */ @@ -68,6 +77,12 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> { private RowKeyEncoderProvider rowKeyEncoderProvider; private HashMap<TblColRef, Boolean> dictsNeedMerging = new HashMap<TblColRef, Boolean>(); + private List<MeasureDesc> measuresDescs; + private MeasureCodec codec; + private Object[] measureObjs; + private Integer[] measureIdxUsingDict; + private ByteBuffer valueBuf; + private Text outputValue; private Boolean checkNeedMerging(TblColRef col) throws IOException { Boolean ret = dictsNeedMerging.get(col); @@ -108,6 +123,22 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> { rowKeySplitter = new RowKeySplitter(sourceCubeSegment, 65, 255); rowKeyEncoderProvider = new RowKeyEncoderProvider(mergedCubeSegment); + + if (cubeDesc.hasMeasureUsingDictionary()) { + measuresDescs = cubeDesc.getMeasures(); + codec = new MeasureCodec(measuresDescs); + measureObjs = new Object[measuresDescs.size()]; + List<Integer> measuresUsingDict = Lists.newArrayList(); + for (int i = 0; i < measuresDescs.size(); i++) { + if (measuresDescs.get(i).getFunction().isTopN()) { + // so far only TopN uses dic + measuresUsingDict.add(i); + } + } + measureIdxUsingDict = measuresUsingDict.toArray(new Integer[measuresUsingDict.size()]); + valueBuf = ByteBuffer.allocate(RowConstants.ROWVALUE_BUFFER_SIZE); + outputValue = new Text(); + } } private static final Pattern JOB_NAME_PATTERN = Pattern.compile("kylin-([0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12})"); @@ -200,6 +231,60 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> { rowkeyEncoder.encode(new ByteArray(newKeyBodyBuf, 0, bufOffset), newKeyBuf); outputKey.set(newKeyBuf.array(), 0, fullKeySize); + // encode measure if it uses dictionary + if (cubeDesc.hasMeasureUsingDictionary()) { + codec.decode(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()), measureObjs); + reEncodeMeasure(measureObjs); + valueBuf.clear(); + codec.encode(measureObjs, valueBuf); + outputValue.set(valueBuf.array(), 0, valueBuf.position()); + value = outputValue; + } + context.write(outputKey, value); } + + private void reEncodeMeasure(Object[] measureObjs) throws IOException, InterruptedException { + int bufOffset = 0; + for (int idx : measureIdxUsingDict) { + // only TopN measure uses dic + TopNCounter<ByteArray> topNCounters = (TopNCounter<ByteArray>) measureObjs[idx]; + + MeasureDesc measureDesc = measuresDescs.get(idx); + String displayCol = measureDesc.getFunction().getParameter().getDisplayColumn().toUpperCase(); + if (StringUtils.isNotEmpty(displayCol)) { + ColumnDesc sourceColumn = cubeDesc.getFactTableDesc().findColumnByName(displayCol); + TblColRef colRef = new TblColRef(sourceColumn); + DictionaryManager dictMgr = DictionaryManager.getInstance(config); + Dictionary<?> sourceDict = dictMgr.getDictionary(sourceCubeSegment.getDictResPath(colRef)); + Dictionary<?> mergedDict = dictMgr.getDictionary(mergedCubeSegment.getDictResPath(colRef)); + + int topNSize = topNCounters.size(); + while (sourceDict.getSizeOfValue() * topNSize > newKeyBodyBuf.length - bufOffset || // + mergedDict.getSizeOfValue() * topNSize > newKeyBodyBuf.length - bufOffset || // + mergedDict.getSizeOfId() * topNSize > newKeyBodyBuf.length - bufOffset) { + byte[] oldBuf = newKeyBodyBuf; + newKeyBodyBuf = new byte[2 * newKeyBodyBuf.length]; + System.arraycopy(oldBuf, 0, newKeyBodyBuf, 0, oldBuf.length); + } + + for (Counter<ByteArray> c : topNCounters) { + int idInSourceDict = BytesUtil.readUnsigned(c.getItem().array(), c.getItem().offset(), c.getItem().length()); + int idInMergedDict; + int size = sourceDict.getValueBytesFromId(idInSourceDict, newKeyBodyBuf, bufOffset); + if (size < 0) { + idInMergedDict = mergedDict.nullId(); + } else { + idInMergedDict = mergedDict.getIdFromValueBytes(newKeyBodyBuf, bufOffset, size); + } + + BytesUtil.writeUnsigned(idInMergedDict, newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId()); + c.getItem().set(newKeyBodyBuf, bufOffset, mergedDict.getSizeOfId()); + bufOffset += mergedDict.getSizeOfId(); + } + } + } + + } + } http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java ---------------------------------------------------------------------- diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java index 2db4ce7..b73fda4 100644 --- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java +++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeDictionaryStep.java @@ -110,16 +110,13 @@ public class MergeDictionaryStep extends AbstractExecutable { DictionaryManager dictMgr = DictionaryManager.getInstance(conf); CubeDesc cubeDesc = cube.getDescriptor(); - for (DimensionDesc dim : cubeDesc.getDimensions()) { - for (TblColRef col : dim.getColumnRefs()) { - if (newSeg.getCubeDesc().getRowkey().isUseDictionary(col)) { - String dictTable = dictMgr.decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col).getTable(); - if (cubeDesc.getFactTable().equalsIgnoreCase(dictTable)) { - colsNeedMeringDict.add(col); - } else { - colsNeedCopyDict.add(col); - } - } + + for (TblColRef col : cubeDesc.getAllColumnsNeedDictionary()) { + String dictTable = dictMgr.decideSourceData(cubeDesc.getModel(), "true", col).getTable(); + if (cubeDesc.getFactTable().equalsIgnoreCase(dictTable)) { + colsNeedMeringDict.add(col); + } else { + colsNeedCopyDict.add(col); } } http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_desc.json ---------------------------------------------------------------------- diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_desc.json index 84cdaf4..fddbb10 100644 --- a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_desc.json +++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_desc.json @@ -88,7 +88,7 @@ "parameter": { "type": "column", "value": "PRICE", - "displaycolumn": "seller_id" + "displaycolumn": "SELLER_ID" }, "returntype": "topn(100)" }, http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_left_join_desc.json ---------------------------------------------------------------------- diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_left_join_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_left_join_desc.json index f7e700d..6aecaae 100644 --- a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_left_join_desc.json +++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_topn_left_join_desc.json @@ -88,7 +88,7 @@ "parameter": { "type": "column", "value": "PRICE", - "displaycolumn": "seller_id" + "displaycolumn": "SELLER_ID" }, "returntype": "topn(100)" }, http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json ---------------------------------------------------------------------- diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json index bd979e0..3f9957b 100644 --- a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json +++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_desc.json @@ -170,7 +170,7 @@ "parameter": { "type": "column", "value": "PRICE", - "displaycolumn": "seller_id" + "displaycolumn": "SELLER_ID" }, "returntype": "topn(100)" }, http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/134960c6/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json ---------------------------------------------------------------------- diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json index dfa62f7..5835a41 100644 --- a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json +++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json @@ -170,7 +170,7 @@ "parameter": { "type": "column", "value": "PRICE", - "displaycolumn": "seller_id" + "displaycolumn": "SELLER_ID" }, "returntype": "topn(100)" },