KYLIN-5052 Clean up unused and out-of-date source code/config (#1702)
* KYLIN-5052 Clean up unused and out-of-date source code/config
- remove unused maven module, include core-dictionary and datasource-sdk
- remove out-of-date UTs
- all source code are not deleted but commented, should deleted them later
* KYLIN-5052 refine unsupported Rest API
* Fix code style issue
diff --git a/build-engine/src/main/java/org/apache/kylin/engine/mr/SortedColumnDFSFile.java b/build-engine/src/main/java/org/apache/kylin/engine/mr/SortedColumnDFSFile.java
index bcf4b98..74122a6 100644
--- a/build-engine/src/main/java/org/apache/kylin/engine/mr/SortedColumnDFSFile.java
+++ b/build-engine/src/main/java/org/apache/kylin/engine/mr/SortedColumnDFSFile.java
@@ -25,8 +25,6 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.dict.ByteComparator;
-import org.apache.kylin.dict.StringBytesConverter;
import org.apache.kylin.metadata.datatype.DataType;
import org.apache.kylin.source.IReadableTable;
import org.slf4j.Logger;
@@ -92,40 +90,41 @@
}
private Comparator<String> getComparatorByType(DataType type) {
- Comparator<String> comparator;
- if (!type.isNumberFamily()) {
- comparator = new ByteComparator<>(new StringBytesConverter());
- } else if (type.isIntegerFamily()) {
- comparator = new Comparator<String>() {
- @Override
- public int compare(String str1, String str2) {
- try {
- Long num1 = Long.parseLong(str1);
- Long num2 = Long.parseLong(str2);
- return num1.compareTo(num2);
- } catch (NumberFormatException e) {
- logger.error("NumberFormatException when parse integer family number.str1:" + str1 + " str2:" + str2);
- e.printStackTrace();
- return 0;
- }
- }
- };
- } else {
- comparator = new Comparator<String>() {
- @Override
- public int compare(String str1, String str2) {
- try {
- Double num1 = Double.parseDouble(str1);
- Double num2 = Double.parseDouble(str2);
- return num1.compareTo(num2);
- } catch (NumberFormatException e) {
- logger.error("NumberFormatException when parse doul family number.str1:" + str1 + " str2:" + str2);
- return 0;
- }
- }
- };
- }
- return comparator;
+ return null;
+// Comparator<String> comparator;
+// if (!type.isNumberFamily()) {
+// comparator = new ByteComparator<>(new StringBytesConverter());
+// } else if (type.isIntegerFamily()) {
+// comparator = new Comparator<String>() {
+// @Override
+// public int compare(String str1, String str2) {
+// try {
+// Long num1 = Long.parseLong(str1);
+// Long num2 = Long.parseLong(str2);
+// return num1.compareTo(num2);
+// } catch (NumberFormatException e) {
+// logger.error("NumberFormatException when parse integer family number.str1:" + str1 + " str2:" + str2);
+// e.printStackTrace();
+// return 0;
+// }
+// }
+// };
+// } else {
+// comparator = new Comparator<String>() {
+// @Override
+// public int compare(String str1, String str2) {
+// try {
+// Double num1 = Double.parseDouble(str1);
+// Double num2 = Double.parseDouble(str2);
+// return num1.compareTo(num2);
+// } catch (NumberFormatException e) {
+// logger.error("NumberFormatException when parse doul family number.str1:" + str1 + " str2:" + str2);
+// return 0;
+// }
+// }
+// };
+// }
+// return comparator;
}
@Override
diff --git a/build/bin/kylin-port-replace-util.sh b/build/bin/kylin-port-replace-util.sh
index ea96791..e5ad078 100755
--- a/build/bin/kylin-port-replace-util.sh
+++ b/build/bin/kylin-port-replace-util.sh
@@ -86,15 +86,6 @@
#replace ports in kylin.properties
new_kylin_port=`expr ${KYLIN_DEFAULT_PORT} + ${OFFSET}`
- #replace kylin.stream.node for Streaming Coordinator
- stream_node="kylin.stream.node=`hostname -f`:$new_kylin_port"
- echo "Using new kylin.stream.node: $stream_node"
- line_count=$(awk '$0 ~ /^kylin.stream.node/ {print $0}' ${KYLIN_CONFIG_FILE} | wc -l)
- if [[ $line_count -eq 0 ]]; then
- echo "kylin.stream.node=`hostname -f`:7070" >> ${KYLIN_CONFIG_FILE}
- fi
- sed -i "s/^kylin\.stream\.node=.*$/$stream_node/g" ${KYLIN_CONFIG_FILE}
-
sed -i "s/#*kylin.server.cluster-servers=\(.*\).*:\(.*\)/kylin.server.cluster-servers=\1:${new_kylin_port}/g" ${KYLIN_CONFIG_FILE}
echo "New kylin port is : ${new_kylin_port}"
diff --git a/build/script/prepare-libs.sh b/build/script/prepare-libs.sh
index 43ce48a..3493f2f 100644
--- a/build/script/prepare-libs.sh
+++ b/build/script/prepare-libs.sh
@@ -33,10 +33,10 @@
cp parquet-assembly/target/parquet-assembly-${version}-job.jar build/lib/kylin-parquet-job-${version}.jar
cp jdbc/target/kylin-jdbc-${version}.jar build/lib/kylin-jdbc-${version}.jar
cp tool-assembly/target/kylin-tool-assembly-${version}-assembly.jar build/tool/kylin-tool-${version}.jar
-cp datasource-sdk/target/kylin-datasource-sdk-${version}-lib.jar build/lib/kylin-datasource-sdk-${version}.jar
+#cp datasource-sdk/target/kylin-datasource-sdk-${version}-lib.jar build/lib/kylin-datasource-sdk-${version}.jar
# Copied file becomes 000 for some env (e.g. my Cygwin)
#chmod 644 build/lib/kylin-job-${version}.jar
chmod 644 build/lib/kylin-jdbc-${version}.jar
chmod 644 build/tool/kylin-tool-${version}.jar
-chmod 644 build/lib/kylin-datasource-sdk-${version}.jar
+#chmod 644 build/lib/kylin-datasource-sdk-${version}.jar
diff --git a/core-cube/pom.xml b/core-cube/pom.xml
index c024c06..91be01b 100644
--- a/core-cube/pom.xml
+++ b/core-cube/pom.xml
@@ -38,10 +38,10 @@
<groupId>org.apache.kylin</groupId>
<artifactId>kylin-core-metadata</artifactId>
</dependency>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-core-dictionary</artifactId>
- </dependency>
+<!-- <dependency>-->
+<!-- <groupId>org.apache.kylin</groupId>-->
+<!-- <artifactId>kylin-core-dictionary</artifactId>-->
+<!-- </dependency>-->
<dependency>
<groupId>org.apache.kylin</groupId>
<artifactId>kylin-shaded-guava</artifactId>
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index 585a37a..838c823 100755
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -27,7 +27,6 @@
import java.util.List;
import java.util.Locale;
import java.util.Map;
-import java.util.Objects;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
@@ -41,19 +40,11 @@
import org.apache.kylin.common.persistence.WriteConflictException;
import org.apache.kylin.common.util.AutoReadWriteLock;
import org.apache.kylin.common.util.AutoReadWriteLock.AutoLock;
-import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.common.util.RandomUtil;
import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.CubeDescTiretreeGlobalDomainDictUtil;
import org.apache.kylin.cube.model.SnapshotTableDesc;
-import org.apache.kylin.dict.DictionaryInfo;
-import org.apache.kylin.dict.DictionaryManager;
-import org.apache.kylin.dict.lookup.ILookupTable;
-import org.apache.kylin.dict.lookup.LookupProviderFactory;
-import org.apache.kylin.dict.lookup.SnapshotManager;
-import org.apache.kylin.dict.lookup.SnapshotTable;
import org.apache.kylin.metadata.TableMetadataManager;
import org.apache.kylin.metadata.cachesync.Broadcaster;
import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
@@ -61,14 +52,11 @@
import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
import org.apache.kylin.metadata.model.DataModelDesc;
import org.apache.kylin.metadata.model.IEngineAware;
-import org.apache.kylin.metadata.model.JoinDesc;
import org.apache.kylin.metadata.model.PartitionDesc;
import org.apache.kylin.metadata.model.SegmentRange;
import org.apache.kylin.metadata.model.SegmentRange.TSRange;
import org.apache.kylin.metadata.model.SegmentStatusEnum;
import org.apache.kylin.metadata.model.Segments;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
import org.apache.kylin.metadata.project.ProjectInstance;
import org.apache.kylin.metadata.project.ProjectManager;
import org.apache.kylin.metadata.project.RealizationEntry;
@@ -77,8 +65,6 @@
import org.apache.kylin.metadata.realization.RealizationRegistry;
import org.apache.kylin.metadata.realization.RealizationStatusEnum;
import org.apache.kylin.metadata.realization.RealizationType;
-import org.apache.kylin.source.IReadableTable;
-import org.apache.kylin.source.SourceManager;
import org.apache.kylin.source.SourcePartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -129,7 +115,6 @@
// a few inner classes to group related methods
private SegmentAssist segAssist = new SegmentAssist();
- private DictionaryAssist dictAssist = new DictionaryAssist();
private Random ran = new Random();
@@ -555,29 +540,6 @@
}
}
- public ILookupTable getLookupTable(CubeSegment cubeSegment, JoinDesc join) {
- String tableName = join.getPKSide().getTableIdentity();
- CubeDesc cubeDesc = cubeSegment.getCubeDesc();
- SnapshotTableDesc snapshotTableDesc = cubeDesc.getSnapshotTableDesc(tableName);
- return getInMemLookupTable(cubeSegment, join, snapshotTableDesc);
- }
-
- private ILookupTable getInMemLookupTable(CubeSegment cubeSegment, JoinDesc join,
- SnapshotTableDesc snapshotTableDesc) {
- String tableName = join.getPKSide().getTableIdentity();
- String snapshotResPath = getSnapshotResPath(cubeSegment, tableName, snapshotTableDesc);
- String[] pkCols = join.getPrimaryKey();
-
- try {
- SnapshotTable snapshot = getSnapshotManager().getSnapshotTable(snapshotResPath);
- TableDesc tableDesc = getMetadataManager().getTableDesc(tableName, cubeSegment.getProject());
- return LookupProviderFactory.getInMemLookupTable(tableDesc, pkCols, snapshot);
- } catch (IOException e) {
- throw new IllegalStateException(
- "Failed to load lookup table " + tableName + " from snapshot " + snapshotResPath, e);
- }
- }
-
private String getSnapshotResPath(CubeSegment cubeSegment, String tableName, SnapshotTableDesc snapshotTableDesc) {
String snapshotResPath;
if (snapshotTableDesc == null || !snapshotTableDesc.isGlobal()) {
@@ -628,13 +590,6 @@
return TableMetadataManager.getInstance(config);
}
- private DictionaryManager getDictionaryManager() {
- return DictionaryManager.getInstance(config);
- }
-
- private SnapshotManager getSnapshotManager() {
- return SnapshotManager.getInstance(config);
- }
private ResourceStore getStore() {
return ResourceStore.getStore(this.config);
@@ -1125,144 +1080,6 @@
// Dictionary/Snapshot related methods
// ============================================================================
- public DictionaryInfo buildDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable)
- throws IOException {
- return dictAssist.buildDictionary(cubeSeg, col, inpTable);
- }
-
- public DictionaryInfo saveDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable,
- Dictionary<String> dict) throws IOException {
- return dictAssist.saveDictionary(cubeSeg, col, inpTable, dict);
- }
-
- /**
- * return null if no dictionary for given column
- */
- public Dictionary<String> getDictionary(CubeSegment cubeSeg, TblColRef col) {
- return dictAssist.getDictionary(cubeSeg, col);
- }
-
- public SnapshotTable buildSnapshotTable(CubeSegment cubeSeg, String lookupTable, String uuid) throws IOException {
- return dictAssist.buildSnapshotTable(cubeSeg, lookupTable, uuid);
- }
-
- private TableMetadataManager getMetadataManager() {
- return TableMetadataManager.getInstance(config);
- }
-
- private class DictionaryAssist {
- public DictionaryInfo buildDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable)
- throws IOException {
- CubeDesc cubeDesc = cubeSeg.getCubeDesc();
- if (!cubeDesc.getAllColumnsNeedDictionaryBuilt().contains(col))
- return null;
-
- String builderClass = cubeDesc.getDictionaryBuilderClass(col);
- DictionaryInfo dictInfo = getDictionaryManager().buildDictionary(col, inpTable, builderClass);
-
- saveDictionaryInfo(cubeSeg, col, dictInfo);
- return dictInfo;
- }
-
- public DictionaryInfo saveDictionary(CubeSegment cubeSeg, TblColRef col, IReadableTable inpTable,
- Dictionary<String> dict) throws IOException {
- CubeDesc cubeDesc = cubeSeg.getCubeDesc();
- if (!cubeDesc.getAllColumnsNeedDictionaryBuilt().contains(col))
- return null;
-
- DictionaryInfo dictInfo = getDictionaryManager().saveDictionary(col, inpTable, dict);
-
- saveDictionaryInfo(cubeSeg, col, dictInfo);
- return dictInfo;
- }
-
- private void saveDictionaryInfo(CubeSegment cubeSeg, TblColRef col, DictionaryInfo dictInfo)
- throws IOException {
- if (dictInfo == null)
- return;
-
- // work on copy instead of cached objects
- CubeInstance cubeCopy = cubeSeg.getCubeInstance().latestCopyForWrite(); // get a latest copy
- CubeSegment segCopy = cubeCopy.getSegmentById(cubeSeg.getUuid());
-
- Dictionary<?> dict = dictInfo.getDictionaryObject();
- segCopy.putDictResPath(col, dictInfo.getResourcePath());
- segCopy.getRowkeyStats().add(new Object[] { col.getIdentity(), dict.getSize(), dict.getSizeOfId() });
-
- CubeUpdate update = new CubeUpdate(cubeCopy);
- update.setToUpdateSegs(segCopy);
- updateCube(update);
- }
-
- /**
- * return null if no dictionary for given column
- */
- @SuppressWarnings("unchecked")
- public Dictionary<String> getDictionary(CubeSegment cubeSeg, TblColRef col) {
- DictionaryInfo info = null;
- String dictResPath = null;
- try {
- DictionaryManager dictMgr = getDictionaryManager();
-
- //tiretree global domain dic
- List<CubeDescTiretreeGlobalDomainDictUtil.GlobalDict> globalDicts = cubeSeg.getCubeDesc().listDomainDict();
- if (!globalDicts.isEmpty()) {
- dictResPath = CubeDescTiretreeGlobalDomainDictUtil.globalReuseDictPath(cubeSeg.getConfig(), col, cubeSeg.getCubeDesc());
- }
-
- if (Objects.isNull(dictResPath)){
- dictResPath = cubeSeg.getDictResPath(col);
- }
-
- if (dictResPath == null)
- return null;
-
- info = dictMgr.getDictionaryInfo(dictResPath);
- if (info == null)
- throw new IllegalStateException("No dictionary found by " + dictResPath
- + ", invalid cube state; cube segment" + cubeSeg + ", col " + col);
- } catch (IOException e) {
- throw new IllegalStateException("Failed to get dictionary for cube segment" + cubeSeg + ", col" + col,
- e);
- }
- return (Dictionary<String>) info.getDictionaryObject();
- }
-
- public SnapshotTable buildSnapshotTable(CubeSegment cubeSeg, String lookupTable, String uuid)
- throws IOException {
- // work on copy instead of cached objects
- CubeInstance cubeCopy = cubeSeg.getCubeInstance().latestCopyForWrite(); // get a latest copy
- CubeSegment segCopy = cubeCopy.getSegmentById(cubeSeg.getUuid());
-
- TableMetadataManager metaMgr = getTableManager();
- SnapshotManager snapshotMgr = getSnapshotManager();
-
- TableDesc tableDesc = new TableDesc(metaMgr.getTableDesc(lookupTable, segCopy.getProject()));
- IReadableTable hiveTable = SourceManager.createReadableTable(tableDesc, uuid);
- SnapshotTable snapshot = snapshotMgr.buildSnapshot(hiveTable, tableDesc, cubeSeg.getConfig());
-
- CubeDesc cubeDesc = cubeSeg.getCubeDesc();
- if (!cubeDesc.isGlobalSnapshotTable(lookupTable)) {
- segCopy.putSnapshotResPath(lookupTable, snapshot.getResourcePath());
- CubeUpdate update = new CubeUpdate(cubeCopy);
- update.setToUpdateSegs(segCopy);
- updateCube(update);
-
- // Update the input cubeSeg after the resource store updated
- cubeSeg.putSnapshotResPath(lookupTable, segCopy.getSnapshotResPath(lookupTable));
- } else {
- CubeUpdate cubeUpdate = new CubeUpdate(cubeCopy);
- Map<String, String> map = Maps.newHashMap();
- map.put(lookupTable, snapshot.getResourcePath());
- cubeUpdate.setUpdateTableSnapshotPath(map);
- updateCube(cubeUpdate);
-
- cubeSeg.getCubeInstance().putSnapshotResPath(lookupTable, snapshot.getResourcePath());
- }
- return snapshot;
- }
- }
-
/**
* To keep "select * from LOOKUP_TABLE" has consistent and latest result, we manually choose
* CubeInstance here to answer such query.
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index c32da70..6f54e1e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -33,7 +33,6 @@
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.annotation.Clarification;
import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.common.util.ShardingHash;
import org.apache.kylin.cube.cuboid.CuboidScheduler;
@@ -371,28 +370,6 @@
this.storageLocationIdentifier = storageLocationIdentifier;
}
- public Map<TblColRef, Dictionary<String>> buildDictionaryMap() {
- Map<TblColRef, Dictionary<String>> result = Maps.newHashMap();
- for (TblColRef col : getCubeDesc().getAllColumnsHaveDictionary()) {
- result.put(col, (Dictionary<String>) getDictionary(col));
- }
- return result;
- }
-
- public Map<TblColRef, Dictionary<String>> buildGlobalDictionaryMap(int globalColumnsSize) {
- Map<TblColRef, Dictionary<String>> result = Maps.newHashMapWithExpectedSize(globalColumnsSize);
- for (TblColRef col : getCubeDesc().getAllGlobalDictColumns()) {
- result.put(col, getDictionary(col));
- }
- return result;
- }
-
- public Dictionary<String> getDictionary(TblColRef col) {
- TblColRef reuseCol = getCubeDesc().getDictionaryReuseColumn(col);
- CubeManager cubeMgr = CubeManager.getInstance(this.getCubeInstance().getConfig());
- return cubeMgr.getDictionary(this, reuseCol);
- }
-
public CubeDimEncMap getDimensionEncodingMap() {
return new CubeDimEncMap(this);
}
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java b/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
index 0815942..c1bafa9 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cli/DictionaryGeneratorCLI.java
@@ -18,170 +18,12 @@
package org.apache.kylin.cube.cli;
-import java.io.IOException;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.ResourceStore;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.model.DimensionDesc;
-import org.apache.kylin.dict.DictionaryInfo;
-import org.apache.kylin.dict.DictionaryInfoSerializer;
-import org.apache.kylin.dict.DictionaryProvider;
-import org.apache.kylin.dict.DistinctColumnValuesProvider;
-import org.apache.kylin.dict.lookup.ILookupTable;
-import org.apache.kylin.dict.lookup.SnapshotTable;
-import org.apache.kylin.dict.lookup.SnapshotTableSerializer;
-import org.apache.kylin.metadata.model.JoinDesc;
-import org.apache.kylin.metadata.model.TableRef;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.IReadableTable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.kylin.common.annotation.Clarification;
-import org.apache.kylin.shaded.com.google.common.collect.Sets;
-
+@Clarification(deprecated=true, msg = "Dimension dict is removed in Kylin 4.")
public class DictionaryGeneratorCLI {
private DictionaryGeneratorCLI() {
}
-
- private static final Logger logger = LoggerFactory.getLogger(DictionaryGeneratorCLI.class);
-
- public static void processSegment(KylinConfig config, String cubeName, String segmentID, String uuid,
- DistinctColumnValuesProvider factTableValueProvider, DictionaryProvider dictProvider) throws IOException {
- CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
- CubeSegment segment = cube.getSegmentById(segmentID);
-
- int retryTime = 0;
- while (retryTime < 3) {
- if (retryTime > 0) {
- logger.info("Rebuild dictionary and snapshot for Cube: {}, Segment: {}, {} times.", cubeName, segmentID,
- retryTime);
- }
-
- processSegment(config, segment, uuid, factTableValueProvider, dictProvider);
-
- if (isAllDictsAndSnapshotsReady(config, cubeName, segmentID)) {
- break;
- }
- retryTime++;
- }
-
- if (retryTime >= 3) {
- logger.error("Not all dictionaries and snapshots ready for cube segment: {}", segmentID);
- } else {
- logger.info("Succeed to build all dictionaries and snapshots for cube segment: {}", segmentID);
- }
- }
-
- private static void processSegment(KylinConfig config, CubeSegment cubeSeg, String uuid,
- DistinctColumnValuesProvider factTableValueProvider, DictionaryProvider dictProvider) throws IOException {
- CubeManager cubeMgr = CubeManager.getInstance(config);
-
- // dictionary
- for (TblColRef col : cubeSeg.getCubeDesc().getAllColumnsNeedDictionaryBuilt()) {
- logger.info("Building dictionary for {}", col);
- IReadableTable inpTable = factTableValueProvider.getDistinctValuesFor(col);
-
- Dictionary<String> preBuiltDict = null;
- if (dictProvider != null) {
- preBuiltDict = dictProvider.getDictionary(col);
- }
-
- if (preBuiltDict != null) {
- logger.debug("Dict for '{}' has already been built, save it", col.getName());
- cubeMgr.saveDictionary(cubeSeg, col, inpTable, preBuiltDict);
- } else {
- logger.debug("Dict for '{}' not pre-built, build it from {}", col.getName(), inpTable);
- cubeMgr.buildDictionary(cubeSeg, col, inpTable);
- }
- }
-
- // snapshot
- Set<String> toSnapshot = Sets.newHashSet();
- Set<TableRef> toCheckLookup = Sets.newHashSet();
- for (DimensionDesc dim : cubeSeg.getCubeDesc().getDimensions()) {
- TableRef table = dim.getTableRef();
- if (cubeSeg.getModel().isLookupTable(table)) {
- // only the snapshot desc is not ext type, need to take snapshot
- toSnapshot.add(table.getTableIdentity());
- toCheckLookup.add(table);
- }
- }
-
- for (String tableIdentity : toSnapshot) {
- logger.info("Building snapshot of {}", tableIdentity);
- cubeMgr.buildSnapshotTable(cubeSeg, tableIdentity, uuid);
- }
-
- CubeInstance updatedCube = cubeMgr.getCube(cubeSeg.getCubeInstance().getName());
- cubeSeg = updatedCube.getSegmentById(cubeSeg.getUuid());
- for (TableRef lookup : toCheckLookup) {
- logger.info("Checking snapshot of {}", lookup);
- try {
- JoinDesc join = cubeSeg.getModel().getJoinsTree().getJoinByPKSide(lookup);
- ILookupTable table = cubeMgr.getLookupTable(cubeSeg, join);
- if (table != null) {
- IOUtils.closeStream(table);
- }
- } catch (Throwable th) {
- throw new RuntimeException(String.format(Locale.ROOT, "Checking snapshot of %s failed.", lookup), th);
- }
- }
- }
-
- private static boolean isAllDictsAndSnapshotsReady(KylinConfig config, String cubeName, String segmentID) {
- CubeInstance cube = CubeManager.getInstance(config).reloadCube(cubeName);
- CubeSegment segment = cube.getSegmentById(segmentID);
- ResourceStore store = ResourceStore.getStore(config);
-
- // check dicts
- logger.info("Begin to check if all dictionaries exist of Segment: {}", segmentID);
- Map<String, String> dictionaries = segment.getDictionaries();
- for (Map.Entry<String, String> entry : dictionaries.entrySet()) {
- String dictResPath = entry.getValue();
- String dictKey = entry.getKey();
- try {
- DictionaryInfo dictInfo = store.getResource(dictResPath, DictionaryInfoSerializer.INFO_SERIALIZER);
- if (dictInfo == null) {
- logger.warn("Dictionary=[key: {}, resource path: {}] doesn't exist in resource store", dictKey,
- dictResPath);
- return false;
- }
- } catch (IOException e) {
- logger.warn("Dictionary=[key: {}, path: {}] failed to check, details: {}", dictKey, dictResPath, e);
- return false;
- }
- }
-
- // check snapshots
- logger.info("Begin to check if all snapshots exist of Segment: {}", segmentID);
- Map<String, String> snapshots = segment.getSnapshots();
- for (Map.Entry<String, String> entry : snapshots.entrySet()) {
- String snapshotKey = entry.getKey();
- String snapshotResPath = entry.getValue();
- try {
- SnapshotTable snapshot = store.getResource(snapshotResPath, SnapshotTableSerializer.INFO_SERIALIZER);
- if (snapshot == null) {
- logger.info("SnapshotTable=[key: {}, resource path: {}] doesn't exist in resource store",
- snapshotKey, snapshotResPath);
- return false;
- }
- } catch (IOException e) {
- logger.warn("SnapshotTable=[key: {}, resource path: {}] failed to check, details: {}", snapshotKey,
- snapshotResPath, e);
- return false;
- }
- }
-
- logger.info("All dictionaries and snapshots exist checking succeed for Cube Segment: {}", segmentID);
- return true;
- }
}
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cli/DumpDictionaryCLI.java b/core-cube/src/main/java/org/apache/kylin/cube/cli/DumpDictionaryCLI.java
index 729a6da..6302aef 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cli/DumpDictionaryCLI.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cli/DumpDictionaryCLI.java
@@ -18,45 +18,10 @@
package org.apache.kylin.cube.cli;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.Date;
-import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.dict.DictionaryInfo;
-import org.apache.kylin.dict.DictionaryInfoSerializer;
+import org.apache.kylin.common.annotation.Clarification;
+@Clarification(deprecated=true, msg = "Dimension dict is removed in Kylin 4.")
public class DumpDictionaryCLI {
- public static void main(String[] args) throws IOException {
- for (String path : args) {
- dump(new File(path));
- }
- }
-
- public static void dump(File f) throws IOException {
- if (f.isDirectory()) {
- File[] files = f.listFiles();
- if (files == null) {
- return;
- }
- for (File c : files)
- dump(c);
- return;
- }
-
- if (f.getName().endsWith(".dict")) {
- DictionaryInfoSerializer ser = new DictionaryInfoSerializer();
- DictionaryInfo dictInfo = ser.deserialize(new DataInputStream(new FileInputStream(f)));
-
- System.out.println("============================================================================");
- System.out.println("File: " + f.getAbsolutePath());
- System.out.println(new Date(dictInfo.getLastModified()));
- System.out.println(JsonUtil.writeValueAsIndentString(dictInfo));
- dictInfo.getDictionaryObject().dump(System.out);
- System.out.println();
- }
- }
}
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java
index 7342a69..64624ca 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java
@@ -36,7 +36,7 @@
import org.apache.kylin.shaded.com.google.common.collect.Maps;
-@Clarification(deprecated = true, msg = "Useless code in Kylin 4")
+@Clarification(deprecated = true, msg = "Dimension dict is removed in Kylin 4 .")
public class CubeDimEncMap implements IDimensionEncodingMap, java.io.Serializable {
private static final Logger logger = LoggerFactory.getLogger(CubeDimEncMap.class);
@@ -83,10 +83,10 @@
@Override
public Dictionary<String> getDictionary(TblColRef col) {
- if (seg == null)
- return dictionaryMap.get(col);
- else
- return seg.getDictionary(col);
+// if (seg == null)
+ return dictionaryMap.get(col);
+// else
+// return seg.getDictionary(col);
}
}
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 c0c6882..1cc31d0 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
@@ -60,8 +60,6 @@
import org.apache.kylin.common.util.JsonUtil;
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.cube.cuboid.CuboidScheduler;
-import org.apache.kylin.dict.GlobalDictionaryBuilder;
-import org.apache.kylin.dict.global.SegmentAppendTrieDictBuilder;
import org.apache.kylin.measure.MeasureType;
import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
import org.apache.kylin.metadata.MetadataConstants;
@@ -1559,27 +1557,27 @@
return null;
}
- public List<TblColRef> getAllGlobalDictColumns() {
- List<TblColRef> globalDictCols = new ArrayList<TblColRef>();
- List<DictionaryDesc> dictionaryDescList = getDictionaries();
-
- if (dictionaryDescList == null) {
- return globalDictCols;
- }
-
- for (DictionaryDesc dictionaryDesc : dictionaryDescList) {
- String cls = dictionaryDesc.getBuilderClass();
- if (GlobalDictionaryBuilder.class.getName().equals(cls)
- || SegmentAppendTrieDictBuilder.class.getName().equals(cls))
- globalDictCols.add(dictionaryDesc.getColumnRef());
- }
- return globalDictCols;
- }
+// public List<TblColRef> getAllGlobalDictColumns() {
+// List<TblColRef> globalDictCols = new ArrayList<TblColRef>();
+// List<DictionaryDesc> dictionaryDescList = getDictionaries();
+//
+// if (dictionaryDescList == null) {
+// return globalDictCols;
+// }
+//
+// for (DictionaryDesc dictionaryDesc : dictionaryDescList) {
+// String cls = dictionaryDesc.getBuilderClass();
+// if (GlobalDictionaryBuilder.class.getName().equals(cls)
+// || SegmentAppendTrieDictBuilder.class.getName().equals(cls))
+// globalDictCols.add(dictionaryDesc.getColumnRef());
+// }
+// return globalDictCols;
+// }
// UHC (ultra high cardinality column): contain the ShardByColumns and the GlobalDictionaryColumns
public List<TblColRef> getAllUHCColumns() {
List<TblColRef> uhcColumns = new ArrayList<>();
- uhcColumns.addAll(getAllGlobalDictColumns());
+// uhcColumns.addAll(getAllGlobalDictColumns());
uhcColumns.addAll(getShardByColumns());
return uhcColumns;
}
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/SnapshotTableDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/SnapshotTableDesc.java
index 30f533b..7f9ee24 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/SnapshotTableDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/SnapshotTableDesc.java
@@ -18,8 +18,6 @@
package org.apache.kylin.cube.model;
-import org.apache.kylin.dict.lookup.SnapshotTable;
-
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
import com.fasterxml.jackson.annotation.JsonProperty;
@@ -33,7 +31,7 @@
private String tableName;
@JsonProperty("storage_type")
- private String storageType = SnapshotTable.STORAGE_TYPE_METASTORE;
+ private String storageType = "metaStore";
@JsonProperty("local_cache_enable")
private boolean enableLocalCache = true;
@@ -65,17 +63,17 @@
this.global = global;
}
- public boolean isExtSnapshotTable() {
- return !SnapshotTable.STORAGE_TYPE_METASTORE.equals(storageType);
- }
-
- public boolean isEnableLocalCache() {
- return enableLocalCache;
- }
-
- public void setEnableLocalCache(boolean enableLocalCache) {
- this.enableLocalCache = enableLocalCache;
- }
+// public boolean isExtSnapshotTable() {
+// return !SnapshotTable.STORAGE_TYPE_METASTORE.equals(storageType);
+// }
+//
+// public boolean isEnableLocalCache() {
+// return enableLocalCache;
+// }
+//
+// public void setEnableLocalCache(boolean enableLocalCache) {
+// this.enableLocalCache = enableLocalCache;
+// }
public static SnapshotTableDesc getCopyOf(SnapshotTableDesc other) {
SnapshotTableDesc copy = new SnapshotTableDesc();
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
index 9023f28..16370f1 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/validation/rule/DictionaryRule.java
@@ -18,20 +18,9 @@
package org.apache.kylin.cube.model.validation.rule;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.lang.StringUtils;
import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.DictionaryDesc;
-import org.apache.kylin.cube.model.RowKeyDesc;
import org.apache.kylin.cube.model.validation.IValidatorRule;
-import org.apache.kylin.cube.model.validation.ResultLevel;
import org.apache.kylin.cube.model.validation.ValidateContext;
-import org.apache.kylin.dict.GlobalDictionaryBuilder;
-import org.apache.kylin.metadata.model.TblColRef;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -47,69 +36,7 @@
public class DictionaryRule implements IValidatorRule<CubeDesc> {
private static final Logger logger = LoggerFactory.getLogger(DictionaryRule.class);
- static final String ERROR_DUPLICATE_DICTIONARY_COLUMN = "Duplicated dictionary specification for column: ";
- static final String ERROR_REUSE_BUILDER_BOTH_SET = "REUSE and BUILDER both set on dictionary for column: ";
- static final String ERROR_REUSE_BUILDER_BOTH_EMPTY = "REUSE and BUILDER both empty on dictionary for column: ";
- static final String ERROR_TRANSITIVE_REUSE = "Transitive REUSE is not allowed for dictionary: ";
- static final String ERROR_GLOBAL_DICTIONNARY_ONLY_MEASURE = "If one column is used for both dimension and precisely count distinct measure, its dimension encoding should not be dict: ";
-
@Override
public void validate(CubeDesc cubeDesc, ValidateContext context) {
- List<DictionaryDesc> dictDescs = cubeDesc.getDictionaries();
- Set<TblColRef> dimensionColumns = cubeDesc.listDimensionColumnsIncludingDerived();
- RowKeyDesc rowKeyDesc = cubeDesc.getRowkey();
-
- if (dictDescs == null || dictDescs.isEmpty()) {
- return;
- }
-
- Set<TblColRef> allDictCols = new HashSet<>();
- Set<TblColRef> baseCols = new HashSet<>(); // col with builder
- List<DictionaryDesc> reuseDictionaries = new ArrayList<>();
-
- // first pass
- for (DictionaryDesc dictDesc : dictDescs) {
- TblColRef dictCol = dictDesc.getColumnRef();
- TblColRef reuseCol = dictDesc.getResuseColumnRef();
- String builderClass = dictDesc.getBuilderClass();
-
- if (!allDictCols.add(dictCol)) {
- context.addResult(ResultLevel.ERROR, ERROR_DUPLICATE_DICTIONARY_COLUMN + dictCol);
- return;
- }
-
- if (reuseCol != null && StringUtils.isNotEmpty(builderClass)) {
- context.addResult(ResultLevel.ERROR, ERROR_REUSE_BUILDER_BOTH_SET + dictCol);
- return;
- }
-
- if (reuseCol == null && StringUtils.isEmpty(builderClass)) {
- if(dictDesc.isDomain()) {
- logger.info("() is tiretree global domain dic", dictCol);
- }else{
- context.addResult(ResultLevel.ERROR, ERROR_REUSE_BUILDER_BOTH_EMPTY + dictCol);
- return;
- }
- }
-
- if (StringUtils.isNotEmpty(builderClass) && builderClass.equalsIgnoreCase(GlobalDictionaryBuilder.class.getName()) && dimensionColumns.contains(dictCol) && rowKeyDesc.isUseDictionary(dictCol)) {
- context.addResult(ResultLevel.ERROR, ERROR_GLOBAL_DICTIONNARY_ONLY_MEASURE + dictCol);
- return;
- }
-
- if (reuseCol != null) {
- reuseDictionaries.add(dictDesc);
- } else {
- baseCols.add(dictCol);
- }
- }
-
- // second pass: check no transitive reuse
- for (DictionaryDesc dictDesc : reuseDictionaries) {
- if (!baseCols.contains(dictDesc.getResuseColumnRef())) {
- context.addResult(ResultLevel.ERROR, ERROR_TRANSITIVE_REUSE + dictDesc.getColumnRef());
- return;
- }
- }
}
}
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java b/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
index c6d0c00..ececb8c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
@@ -18,31 +18,19 @@
package org.apache.kylin.cube.util;
-import java.io.IOException;
-import java.util.HashMap;
import java.util.List;
-import java.util.Locale;
import java.util.Map;
import java.util.Set;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.cube.model.CubeJoinedFlatTableEnrich;
-import org.apache.kylin.dict.DictionaryGenerator;
-import org.apache.kylin.dict.DictionaryInfo;
-import org.apache.kylin.dict.DictionaryManager;
-import org.apache.kylin.dict.IterableDictionaryValueEnumerator;
import org.apache.kylin.measure.hllc.HLLCounter;
import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.IReadableTable;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.kylin.shaded.com.google.common.collect.HashMultimap;
import org.apache.kylin.shaded.com.google.common.collect.Maps;
import org.apache.kylin.shaded.com.google.common.hash.HashFunction;
import org.apache.kylin.shaded.com.google.common.hash.Hasher;
@@ -107,61 +95,61 @@
return result;
}
- public static Map<TblColRef, Dictionary<String>> buildDictionary(final CubeInstance cubeInstance,
- Iterable<List<String>> recordList) throws IOException {
- final List<TblColRef> columnsNeedToBuildDictionary = cubeInstance.getDescriptor()
- .listDimensionColumnsExcludingDerived(true);
- final HashMap<Integer, TblColRef> tblColRefMap = Maps.newHashMap();
- int index = 0;
- for (TblColRef column : columnsNeedToBuildDictionary) {
- tblColRefMap.put(index++, column);
- }
+// public static Map<TblColRef, Dictionary<String>> buildDictionary(final CubeInstance cubeInstance,
+// Iterable<List<String>> recordList) throws IOException {
+// final List<TblColRef> columnsNeedToBuildDictionary = cubeInstance.getDescriptor()
+// .listDimensionColumnsExcludingDerived(true);
+// final HashMap<Integer, TblColRef> tblColRefMap = Maps.newHashMap();
+// int index = 0;
+// for (TblColRef column : columnsNeedToBuildDictionary) {
+// tblColRefMap.put(index++, column);
+// }
+//
+// HashMap<TblColRef, Dictionary<String>> result = Maps.newHashMap();
+//
+// HashMultimap<TblColRef, String> valueMap = HashMultimap.create();
+// for (List<String> row : recordList) {
+// for (int i = 0; i < row.size(); i++) {
+// String cell = row.get(i);
+// if (tblColRefMap.containsKey(i)) {
+// valueMap.put(tblColRefMap.get(i), cell);
+// }
+// }
+// }
+// for (TblColRef tblColRef : valueMap.keySet()) {
+// Set<String> values = valueMap.get(tblColRef);
+// Dictionary<String> dict = DictionaryGenerator.buildDictionary(tblColRef.getType(),
+// new IterableDictionaryValueEnumerator(values));
+// result.put(tblColRef, dict);
+// }
+// return result;
+// }
- HashMap<TblColRef, Dictionary<String>> result = Maps.newHashMap();
-
- HashMultimap<TblColRef, String> valueMap = HashMultimap.create();
- for (List<String> row : recordList) {
- for (int i = 0; i < row.size(); i++) {
- String cell = row.get(i);
- if (tblColRefMap.containsKey(i)) {
- valueMap.put(tblColRefMap.get(i), cell);
- }
- }
- }
- for (TblColRef tblColRef : valueMap.keySet()) {
- Set<String> values = valueMap.get(tblColRef);
- Dictionary<String> dict = DictionaryGenerator.buildDictionary(tblColRef.getType(),
- new IterableDictionaryValueEnumerator(values));
- result.put(tblColRef, dict);
- }
- return result;
- }
-
- @SuppressWarnings("unchecked")
- public static Map<TblColRef, Dictionary<String>> writeDictionary(CubeSegment cubeSegment,
- Map<TblColRef, Dictionary<String>> dictionaryMap, long startOffset, long endOffset) {
- Map<TblColRef, Dictionary<String>> realDictMap = Maps.newHashMap();
-
- for (Map.Entry<TblColRef, Dictionary<String>> entry : dictionaryMap.entrySet()) {
- final TblColRef tblColRef = entry.getKey();
- final Dictionary<String> dictionary = entry.getValue();
- IReadableTable.TableSignature signature = new IReadableTable.TableSignature();
- signature.setLastModifiedTime(System.currentTimeMillis());
- signature.setPath(String.format(Locale.ROOT, "streaming_%s_%s", startOffset, endOffset));
- signature.setSize(endOffset - startOffset);
- DictionaryInfo dictInfo = new DictionaryInfo(tblColRef.getColumnDesc(), tblColRef.getDatatype(), signature);
- logger.info("writing dictionary for TblColRef:" + tblColRef.toString());
- DictionaryManager dictionaryManager = DictionaryManager.getInstance(cubeSegment.getCubeDesc().getConfig());
- try {
- DictionaryInfo realDict = dictionaryManager.trySaveNewDict(dictionary, dictInfo);
- cubeSegment.putDictResPath(tblColRef, realDict.getResourcePath());
- realDictMap.put(tblColRef, (Dictionary<String>) realDict.getDictionaryObject());
- } catch (IOException e) {
- throw new RuntimeException("error save dictionary for column:" + tblColRef, e);
- }
- }
-
- return realDictMap;
- }
+// @SuppressWarnings("unchecked")
+// public static Map<TblColRef, Dictionary<String>> writeDictionary(CubeSegment cubeSegment,
+// Map<TblColRef, Dictionary<String>> dictionaryMap, long startOffset, long endOffset) {
+// Map<TblColRef, Dictionary<String>> realDictMap = Maps.newHashMap();
+//
+// for (Map.Entry<TblColRef, Dictionary<String>> entry : dictionaryMap.entrySet()) {
+// final TblColRef tblColRef = entry.getKey();
+// final Dictionary<String> dictionary = entry.getValue();
+// IReadableTable.TableSignature signature = new IReadableTable.TableSignature();
+// signature.setLastModifiedTime(System.currentTimeMillis());
+// signature.setPath(String.format(Locale.ROOT, "streaming_%s_%s", startOffset, endOffset));
+// signature.setSize(endOffset - startOffset);
+// DictionaryInfo dictInfo = new DictionaryInfo(tblColRef.getColumnDesc(), tblColRef.getDatatype(), signature);
+// logger.info("writing dictionary for TblColRef:" + tblColRef.toString());
+// DictionaryManager dictionaryManager = DictionaryManager.getInstance(cubeSegment.getCubeDesc().getConfig());
+// try {
+// DictionaryInfo realDict = dictionaryManager.trySaveNewDict(dictionary, dictInfo);
+// cubeSegment.putDictResPath(tblColRef, realDict.getResourcePath());
+// realDictMap.put(tblColRef, (Dictionary<String>) realDict.getDictionaryObject());
+// } catch (IOException e) {
+// throw new RuntimeException("error save dictionary for column:" + tblColRef, e);
+// }
+// }
+//
+// return realDictMap;
+// }
}
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/common/RowKeySplitterTest.java b/core-cube/src/test/java/org/apache/kylin/cube/common/RowKeySplitterTest.java
index e4a426d..2350852 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/common/RowKeySplitterTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/common/RowKeySplitterTest.java
@@ -25,8 +25,10 @@
import org.apache.kylin.cube.CubeManager;
import org.junit.After;
import org.junit.Before;
+import org.junit.Ignore;
import org.junit.Test;
+@Ignore
public class RowKeySplitterTest extends LocalFileMetadataTestCase {
@Before
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 142852e..c304242 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
@@ -34,6 +34,7 @@
import org.junit.Ignore;
import org.junit.Test;
+@Ignore
public class RowKeyDecoderTest extends LocalFileMetadataTestCase {
@Before
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 335dab3..a4666e6 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
@@ -34,6 +34,7 @@
import org.junit.Ignore;
import org.junit.Test;
+@Ignore
public class RowKeyEncoderTest extends LocalFileMetadataTestCase {
@Before
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
deleted file mode 100644
index de436d0..0000000
--- a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.cube.model.validation.rule;
-
-import static org.apache.kylin.cube.model.validation.rule.DictionaryRule.ERROR_DUPLICATE_DICTIONARY_COLUMN;
-import static org.apache.kylin.cube.model.validation.rule.DictionaryRule.ERROR_GLOBAL_DICTIONNARY_ONLY_MEASURE;
-import static org.apache.kylin.cube.model.validation.rule.DictionaryRule.ERROR_REUSE_BUILDER_BOTH_EMPTY;
-import static org.apache.kylin.cube.model.validation.rule.DictionaryRule.ERROR_REUSE_BUILDER_BOTH_SET;
-import static org.apache.kylin.cube.model.validation.rule.DictionaryRule.ERROR_TRANSITIVE_REUSE;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.kylin.shaded.com.google.common.collect.Lists;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.JsonUtil;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.model.CubeDesc;
-import org.apache.kylin.cube.model.DictionaryDesc;
-import org.apache.kylin.cube.model.validation.ValidateContext;
-import org.apache.kylin.dict.GlobalDictionaryBuilder;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class DictionaryRuleTest extends LocalFileMetadataTestCase {
- private static KylinConfig config;
-
- @Before
- public void setUp() throws Exception {
- this.createTestMetadata();
- config = KylinConfig.getInstanceFromEnv();
- }
-
- @After
- public void after() throws Exception {
- this.cleanupTestMetadata();
- }
-
- @Test
- public void testGoodDesc() throws IOException {
- DictionaryRule rule = new DictionaryRule();
-
- for (File f : new File(LocalFileMetadataTestCase.LOCALMETA_TEST_DATA + "/cube_desc/").listFiles()) {
- if (!f.getName().endsWith("json")) {
- continue;
- }
- CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
- desc.init(config);
- ValidateContext vContext = new ValidateContext();
- rule.validate(desc, vContext);
- assertTrue(vContext.getResults().length == 0);
- }
- }
-
- @Test
- public void testBadDesc() throws IOException {
- testDictionaryDesc(ERROR_DUPLICATE_DICTIONARY_COLUMN, DictionaryDesc.create("ORDER_ID", null, "FakeBuilderClass"));
- testDictionaryDesc(ERROR_DUPLICATE_DICTIONARY_COLUMN, DictionaryDesc.create("ORDER_ID", null, GlobalDictionaryBuilder.class.getName()));
- }
-
- @Test
- public void testBadDesc2() throws IOException {
- testDictionaryDesc(ERROR_REUSE_BUILDER_BOTH_SET, DictionaryDesc.create("lstg_site_id", "SITE_NAME", "FakeBuilderClass"));
- }
-
- @Test
- public void testBadDesc3() throws IOException {
- testDictionaryDesc(ERROR_REUSE_BUILDER_BOTH_EMPTY, DictionaryDesc.create("lstg_site_id", null, null));
- }
-
- @Test
- public void testBadDesc4() throws IOException {
- testDictionaryDesc(ERROR_TRANSITIVE_REUSE,
- DictionaryDesc.create("lstg_site_id", "SELLER_ID", null),
- DictionaryDesc.create("price", "lstg_site_id", null));
- }
-
- @Test
- public void testBadDesc5() throws IOException {
- testDictionaryDesc(ERROR_GLOBAL_DICTIONNARY_ONLY_MEASURE,
- DictionaryDesc.create("CATEG_LVL2_NAME", null, GlobalDictionaryBuilder.class.getName()));
- }
-
- @Test
- public void testGoodDesc2() throws IOException {
- testDictionaryDesc(null, DictionaryDesc.create("SELLER_ID", null, GlobalDictionaryBuilder.class.getName()));
- }
-
- private void testDictionaryDesc(String expectMessage, DictionaryDesc... descs) throws IOException {
- DictionaryRule rule = new DictionaryRule();
- File f = new File(LocalFileMetadataTestCase.LOCALMETA_TEST_DATA + "/cube_desc/test_kylin_cube_without_slr_left_join_desc.json");
- CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
-
- List<DictionaryDesc> newDicts = Lists.newArrayList(desc.getDictionaries());
- for (DictionaryDesc dictDesc : descs) {
- newDicts.add(dictDesc);
- }
- desc.setDictionaries(newDicts);
-
- desc.init(config);
- ValidateContext vContext = new ValidateContext();
- rule.validate(desc, vContext);
-
- if (expectMessage == null) {
- assertTrue(vContext.getResults().length == 0);
- } else {
- assertTrue(vContext.getResults().length == 1);
- String actualMessage = vContext.getResults()[0].getMessage();
- assertTrue(actualMessage.startsWith(expectMessage));
- }
- }
-}
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
index c9ac223..d6955bf 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
@@ -29,11 +29,13 @@
import org.apache.kylin.dimension.IntegerDimEnc;
import org.apache.kylin.dimension.OneMoreByteVLongDimEnc;
import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Test;
import org.apache.kylin.shaded.com.google.common.collect.Lists;
import org.apache.kylin.shaded.com.google.common.collect.Ordering;
+@Ignore
public class DimEncodingPreserveOrderTest {
private static List<long[]> successValue;
private static List<long[]> failValue;
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/GTScanReqSerDerTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/GTScanReqSerDerTest.java
index 1ae229a..1acc19a 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/GTScanReqSerDerTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/GTScanReqSerDerTest.java
@@ -34,7 +34,9 @@
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
+import org.junit.Ignore;
+@Ignore
public class GTScanReqSerDerTest extends LocalFileMetadataTestCase {
private ByteBuffer buffer = ByteBuffer.allocate(BytesSerializer.SERIALIZE_BUFFER_SIZE);
diff --git a/core-dictionary/pom.xml b/core-dictionary/pom.xml
deleted file mode 100644
index 48841c1..0000000
--- a/core-dictionary/pom.xml
+++ /dev/null
@@ -1,79 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one
- or more contributor license agreements. See the NOTICE file
- distributed with this work for additional information
- regarding copyright ownership. The ASF licenses this file
- to you under the Apache License, Version 2.0 (the
- "License"); you may not use this file except in compliance
- with the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <artifactId>kylin-core-dictionary</artifactId>
- <packaging>jar</packaging>
- <name>Apache Kylin - Core Dictionary</name>
- <description>Apache Kylin - Core Dictionary</description>
-
- <parent>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin</artifactId>
- <version>4.0.0-SNAPSHOT</version>
- </parent>
-
- <dependencies>
-
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-core-metadata</artifactId>
- </dependency>
-
- <dependency>
- <groupId>org.rocksdb</groupId>
- <artifactId>rocksdbjni</artifactId>
- </dependency>
-
- <!-- Env & Test -->
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-core-common</artifactId>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>com.google.guava</groupId>
- <artifactId>guava-testlib</artifactId>
- <version>${guava-testlib.version}</version>
- <scope>test</scope>
- </dependency>
-
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-surefire-plugin</artifactId>
- <configuration>
- <skip>${skipRunIt}</skip>
- </configuration>
- </plugin>
- </plugins>
- </build>
-
-</project>
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
index 3adbb8e..fcc2c98 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeSegmentScanner.java
@@ -24,17 +24,18 @@
import java.util.List;
import java.util.Set;
+import org.apache.kylin.common.annotation.Clarification;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.dict.BuiltInFunctionTransformer;
+//import org.apache.kylin.dict.BuiltInFunctionTransformer;
import org.apache.kylin.gridtable.GTInfo;
import org.apache.kylin.gridtable.GTRecord;
import org.apache.kylin.gridtable.GTScanRequest;
import org.apache.kylin.metadata.expression.TupleExpression;
-import org.apache.kylin.metadata.filter.ITupleFilterTransformer;
-import org.apache.kylin.metadata.filter.StringCodeSystem;
+//import org.apache.kylin.metadata.filter.ITupleFilterTransformer;
+//import org.apache.kylin.metadata.filter.StringCodeSystem;
import org.apache.kylin.metadata.filter.TupleFilter;
-import org.apache.kylin.metadata.filter.TupleFilterSerializer;
+//import org.apache.kylin.metadata.filter.TupleFilterSerializer;
import org.apache.kylin.metadata.model.DynamicFunctionDesc;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.TblColRef;
@@ -42,50 +43,22 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@Clarification(deprecated = true, msg = "Only for HBase storage")
public class CubeSegmentScanner implements Iterable<GTRecord> {
private static final Logger logger = LoggerFactory.getLogger(CubeSegmentScanner.class);
- final CubeSegment cubeSeg;
- final ScannerWorker scanner;
- final Cuboid cuboid;
+ CubeSegment cubeSeg;
+ ScannerWorker scanner;
+ Cuboid cuboid;
- final GTScanRequest scanRequest;
+ GTScanRequest scanRequest;
public CubeSegmentScanner(CubeSegment cubeSeg, Cuboid cuboid, Set<TblColRef> dimensions, //
Set<TblColRef> groups, List<TblColRef> dynGroups, List<TupleExpression> dynGroupExprs, //
Collection<FunctionDesc> metrics, List<DynamicFunctionDesc> dynFuncs, //
TupleFilter originalfilter, TupleFilter havingFilter, StorageContext context) {
- logger.info("Init CubeSegmentScanner for segment {}", cubeSeg.getName());
-
- this.cuboid = cuboid;
- this.cubeSeg = cubeSeg;
-
- //the filter might be changed later in this CubeSegmentScanner (In ITupleFilterTransformer)
- //to avoid issues like in https://issues.apache.org/jira/browse/KYLIN-1954, make sure each CubeSegmentScanner
- //is working on its own copy
- byte[] serialize = TupleFilterSerializer.serialize(originalfilter, StringCodeSystem.INSTANCE);
- TupleFilter filter = TupleFilterSerializer.deserialize(serialize, StringCodeSystem.INSTANCE);
-
- // translate FunctionTupleFilter to IN clause
- ITupleFilterTransformer translator = new BuiltInFunctionTransformer(cubeSeg.getDimensionEncodingMap());
- filter = translator.transform(filter);
-
- CubeScanRangePlanner scanRangePlanner;
- try {
- scanRangePlanner = new CubeScanRangePlanner(cubeSeg, cuboid, filter, dimensions, groups, dynGroups,
- dynGroupExprs, metrics, dynFuncs, havingFilter, context);
- } catch (RuntimeException e) {
- throw e;
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
-
- scanRequest = scanRangePlanner.planScanRequest();
-
- String gtStorage = ((GTCubeStorageQueryBase) context.getStorageQuery()).getGTStorage();
- scanner = new ScannerWorker(cubeSeg, cuboid, scanRequest, gtStorage, context);
}
public boolean isSegmentSkipped() {
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeTupleConverter.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeTupleConverter.java
index 4fb71bb..7896516 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeTupleConverter.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/CubeTupleConverter.java
@@ -26,22 +26,17 @@
import java.util.Set;
import java.util.TimeZone;
-import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.annotation.Clarification;
import org.apache.kylin.common.util.Array;
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.common.util.Dictionary;
-import org.apache.kylin.cube.CubeManager;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
import org.apache.kylin.cube.model.RowKeyColDesc;
import org.apache.kylin.cube.model.RowKeyDesc;
-import org.apache.kylin.dict.lookup.ILookupTable;
import org.apache.kylin.dimension.TimeDerivedColumnType;
import org.apache.kylin.measure.MeasureType;
import org.apache.kylin.measure.MeasureType.IAdvMeasureFiller;
import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.JoinDesc;
import org.apache.kylin.metadata.model.TblColRef;
import org.apache.kylin.metadata.tuple.Tuple;
import org.apache.kylin.metadata.tuple.TupleInfo;
@@ -49,11 +44,8 @@
import org.slf4j.LoggerFactory;
import org.apache.kylin.shaded.com.google.common.collect.Lists;
-import org.apache.kylin.shaded.com.google.common.collect.Maps;
-/**
- * Convert Object[] (decoded GTRecord) to tuple
- */
+@Clarification(deprecated = true, msg = "Only for HBase storage")
public class CubeTupleConverter implements ITupleConverter {
private static final Logger logger = LoggerFactory.getLogger(CubeTupleConverter.class);
@@ -69,7 +61,6 @@
public final List<IAdvMeasureFiller> advMeasureFillers;
public final List<Integer> advMeasureIndexInGTValues;
- private List<ILookupTable> usedLookupTables;
final Set<Integer> timestampColumn = new HashSet<>();
String eventTimezone;
@@ -96,7 +87,7 @@
advMeasureFillers = Lists.newArrayListWithCapacity(1);
advMeasureIndexInGTValues = Lists.newArrayListWithCapacity(1);
- usedLookupTables = Lists.newArrayList();
+// usedLookupTables = Lists.newArrayList();
eventTimezone = cubeSeg.getConfig().getStreamingDerivedTimeTimezone();
autoJustByTimezone = eventTimezone.length() > 0
&& cubeSeg.getCubeDesc().getModel().getRootFactTable().getTableDesc().isStreamingTable();
@@ -132,8 +123,8 @@
MeasureType<?> measureType = metric.getMeasureType();
if (measureType.needAdvancedTupleFilling()) {
- Map<TblColRef, Dictionary<String>> dictionaryMap = buildDictionaryMap(measureType.getColumnsNeedDictionary(metric));
- advMeasureFillers.add(measureType.getAdvancedTupleFiller(metric, returnTupleInfo, dictionaryMap));
+// Map<TblColRef, Dictionary<String>> dictionaryMap = buildDictionaryMap(measureType.getColumnsNeedDictionary(metric));
+// advMeasureFillers.add(measureType.getAdvancedTupleFiller(metric, returnTupleInfo, dictionaryMap));
advMeasureIndexInGTValues.add(i);
} else {
measureTypes[i] = measureType;
@@ -158,13 +149,13 @@
}
// load only needed dictionaries
- private Map<TblColRef, Dictionary<String>> buildDictionaryMap(List<TblColRef> columnsNeedDictionary) {
- Map<TblColRef, Dictionary<String>> result = Maps.newHashMap();
- for (TblColRef col : columnsNeedDictionary) {
- result.put(col, cubeSeg.getDictionary(col));
- }
- return result;
- }
+// private Map<TblColRef, Dictionary<String>> buildDictionaryMap(List<TblColRef> columnsNeedDictionary) {
+// Map<TblColRef, Dictionary<String>> result = Maps.newHashMap();
+// for (TblColRef col : columnsNeedDictionary) {
+// result.put(col, cubeSeg.getDictionary(col));
+// }
+// return result;
+// }
@Override
public List<IAdvMeasureFiller> translateResult(Object[] gtValues, Tuple tuple) {
@@ -247,13 +238,13 @@
@Override
public void close() throws IOException {
- for (ILookupTable usedLookupTable : usedLookupTables) {
- try {
- usedLookupTable.close();
- } catch (Exception e) {
- logger.error("error when close lookup table:" + usedLookupTable);
- }
- }
+// for (ILookupTable usedLookupTable : usedLookupTables) {
+// try {
+// usedLookupTable.close();
+// } catch (Exception e) {
+// logger.error("error when close lookup table:" + usedLookupTable);
+// }
+// }
}
protected interface IDerivedColumnFiller {
@@ -282,44 +273,44 @@
switch (deriveInfo.type) {
case LOOKUP:
return new IDerivedColumnFiller() {
- ILookupTable lookupTable = getAndAddLookupTable(cubeSeg, deriveInfo.join);
- int[] derivedColIdx = initDerivedColIdx();
- Array<String> lookupKey = new Array<String>(new String[hostTmpIdx.length]);
-
- private int[] initDerivedColIdx() {
- int[] idx = new int[deriveInfo.columns.length];
- for (int i = 0; i < idx.length; i++) {
- idx[i] = deriveInfo.columns[i].getColumnDesc().getZeroBasedIndex();
- }
- return idx;
- }
+// ILookupTable lookupTable = getAndAddLookupTable(cubeSeg, deriveInfo.join);
+// int[] derivedColIdx = initDerivedColIdx();
+// Array<String> lookupKey = new Array<String>(new String[hostTmpIdx.length]);
+//
+// private int[] initDerivedColIdx() {
+// int[] idx = new int[deriveInfo.columns.length];
+// for (int i = 0; i < idx.length; i++) {
+// idx[i] = deriveInfo.columns[i].getColumnDesc().getZeroBasedIndex();
+// }
+// return idx;
+// }
@Override
public void fillDerivedColumns(Object[] gtValues, Tuple tuple) {
- for (int i = 0; i < hostTmpIdx.length; i++) {
- lookupKey.data[i] = CubeTupleConverter.toString(gtValues[hostTmpIdx[i]]);
- // if the primary key of lookup table is date time type, do this change in case of data type inconsistency
- if (lookupKey.data[i] != null && deriveInfo.join.getPrimaryKeyColumns()[i].getType().isDateTimeFamily()) {
- lookupKey.data[i] = String.valueOf(DateFormat.stringToMillis(lookupKey.data[i]));
- }
- }
-
- String[] lookupRow = lookupTable.getRow(lookupKey);
-
- if (lookupRow != null) {
- for (int i = 0; i < derivedTupleIdx.length; i++) {
- if (derivedTupleIdx[i] >= 0) {
- String value = lookupRow[derivedColIdx[i]];
- tuple.setDimensionValue(derivedTupleIdx[i], value);
- }
- }
- } else {
- for (int i = 0; i < derivedTupleIdx.length; i++) {
- if (derivedTupleIdx[i] >= 0) {
- tuple.setDimensionValue(derivedTupleIdx[i], null);
- }
- }
- }
+// for (int i = 0; i < hostTmpIdx.length; i++) {
+// lookupKey.data[i] = CubeTupleConverter.toString(gtValues[hostTmpIdx[i]]);
+// // if the primary key of lookup table is date time type, do this change in case of data type inconsistency
+// if (lookupKey.data[i] != null && deriveInfo.join.getPrimaryKeyColumns()[i].getType().isDateTimeFamily()) {
+// lookupKey.data[i] = String.valueOf(DateFormat.stringToMillis(lookupKey.data[i]));
+// }
+// }
+//
+// String[] lookupRow = lookupTable.getRow(lookupKey);
+//
+// if (lookupRow != null) {
+// for (int i = 0; i < derivedTupleIdx.length; i++) {
+// if (derivedTupleIdx[i] >= 0) {
+// String value = lookupRow[derivedColIdx[i]];
+// tuple.setDimensionValue(derivedTupleIdx[i], value);
+// }
+// }
+// } else {
+// for (int i = 0; i < derivedTupleIdx.length; i++) {
+// if (derivedTupleIdx[i] >= 0) {
+// tuple.setDimensionValue(derivedTupleIdx[i], null);
+// }
+// }
+// }
}
};
case PK_FK:
@@ -345,11 +336,11 @@
return -1;
}
- public ILookupTable getAndAddLookupTable(CubeSegment cubeSegment, JoinDesc join) {
- ILookupTable lookupTable = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getLookupTable(cubeSegment, join);
- usedLookupTables.add(lookupTable);
- return lookupTable;
- }
+// public ILookupTable getAndAddLookupTable(CubeSegment cubeSegment, JoinDesc join) {
+// ILookupTable lookupTable = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getLookupTable(cubeSegment, join);
+// usedLookupTables.add(lookupTable);
+// return lookupTable;
+// }
private static String toString(Object o) {
return o == null ? null : o.toString();
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java
index 804ce3f..35293d8 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java
@@ -18,7 +18,6 @@
package org.apache.kylin.storage.gtrecord;
-import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
@@ -31,17 +30,14 @@
import org.apache.kylin.common.QueryContextFacade;
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.cube.RawQueryLastHacker;
-import org.apache.kylin.cube.common.SegmentPruner;
import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping;
import org.apache.kylin.cube.gridtable.CuboidToGridTableMappingExt;
import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
import org.apache.kylin.cube.model.RowKeyColDesc;
-import org.apache.kylin.dict.lookup.ILookupTable;
import org.apache.kylin.gridtable.StorageLimitLevel;
import org.apache.kylin.measure.MeasureType;
import org.apache.kylin.measure.bitmap.BitmapMeasureType;
@@ -86,25 +82,7 @@
@Override
public ITupleIterator search(StorageContext context, SQLDigest sqlDigest, TupleInfo returnTupleInfo) {
- GTCubeStorageQueryRequest request = getStorageQueryRequest(context, sqlDigest, returnTupleInfo);
-
- List<CubeSegmentScanner> scanners = Lists.newArrayList();
- SegmentPruner segPruner = new SegmentPruner(sqlDigest.filter);
- for (CubeSegment cubeSeg : segPruner.listSegmentsForQuery(cubeInstance)) {
- CubeSegmentScanner scanner = new CubeSegmentScanner(cubeSeg, request.getCuboid(), request.getDimensions(), //
- request.getGroups(), request.getDynGroups(), request.getDynGroupExprs(), //
- request.getMetrics(), request.getDynFuncs(), //
- request.getFilter(), request.getHavingFilter(), request.getContext());
- if (!scanner.isSegmentSkipped())
- scanners.add(scanner);
- }
-
- if (scanners.isEmpty())
- return ITupleIterator.EMPTY_TUPLE_ITERATOR;
-
- return new SequentialCubeTupleIterator(scanners, request.getCuboid(), request.getDimensions(),
- request.getDynGroups(), request.getGroups(), request.getMetrics(), returnTupleInfo,
- request.getContext(), sqlDigest);
+ throw new UnsupportedOperationException("Removed in Kylin 4.0 .");
}
public GTCubeStorageQueryRequest getStorageQueryRequest(StorageContext context, SQLDigest sqlDigest,
@@ -383,16 +361,16 @@
return compf;
DeriveInfo hostInfo = cubeDesc.getHostInfo(derived);
- ILookupTable lookup = cubeDesc.getHostInfo(derived).type == CubeDesc.DeriveType.PK_FK ? null
- : getLookupStringTableForDerived(derived, hostInfo);
- Pair<TupleFilter, Boolean> translated = DerivedFilterTranslator.translate(lookup, hostInfo, compf);
- try {
- if (lookup != null) {
- lookup.close();
- }
- } catch (IOException e) {
- logger.error("error when close lookup table.", e);
- }
+// ILookupTable lookup = cubeDesc.getHostInfo(derived).type == CubeDesc.DeriveType.PK_FK ? null
+// : getLookupStringTableForDerived(derived, hostInfo);
+ Pair<TupleFilter, Boolean> translated = DerivedFilterTranslator.translate(hostInfo, compf);
+// try {
+// if (lookup != null) {
+// lookup.close();
+// }
+// } catch (IOException e) {
+// logger.error("error when close lookup table.", e);
+// }
TupleFilter translatedFilter = translated.getFirst();
boolean loosened = translated.getSecond();
if (loosened) {
@@ -401,12 +379,12 @@
return translatedFilter;
}
- @SuppressWarnings("unchecked")
- protected ILookupTable getLookupStringTableForDerived(TblColRef derived, DeriveInfo hostInfo) {
- CubeManager cubeMgr = CubeManager.getInstance(this.cubeInstance.getConfig());
- CubeSegment seg = cubeInstance.getLatestReadySegment();
- return cubeMgr.getLookupTable(seg, hostInfo.join);
- }
+// @SuppressWarnings("unchecked")
+// protected ILookupTable getLookupStringTableForDerived(TblColRef derived, DeriveInfo hostInfo) {
+// CubeManager cubeMgr = CubeManager.getInstance(this.cubeInstance.getConfig());
+// CubeSegment seg = cubeInstance.getLatestReadySegment();
+// return cubeMgr.getLookupTable(seg, hostInfo.join);
+// }
private void collectColumnsRecursively(TupleFilter filter, Set<TblColRef> collector) {
if (filter == null)
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
index f51ca62..0bb92f9 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
@@ -24,6 +24,7 @@
import java.util.NoSuchElementException;
import java.util.Set;
+import org.apache.kylin.common.annotation.Clarification;
import org.apache.kylin.shaded.com.google.common.collect.UnmodifiableIterator;
import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping;
@@ -43,6 +44,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+@Clarification(deprecated = true, msg = "Only for HBase storage")
public class SegmentCubeTupleIterator implements ITupleIterator {
private static final Logger logger = LoggerFactory.getLogger(SegmentCubeTupleIterator.class);
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
index c17a9fb..ca9d5c3 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SequentialCubeTupleIterator.java
@@ -25,6 +25,7 @@
import java.util.Set;
import java.util.TreeSet;
+import org.apache.kylin.common.annotation.Clarification;
import org.apache.kylin.shaded.com.google.common.collect.Sets;
import org.apache.kylin.common.QueryContextFacade;
import org.apache.kylin.cube.cuboid.Cuboid;
@@ -42,6 +43,7 @@
import org.apache.kylin.shaded.com.google.common.collect.Iterators;
import org.apache.kylin.shaded.com.google.common.collect.Lists;
+@Clarification(deprecated = true, msg = "Only for HBase storage")
public class SequentialCubeTupleIterator implements ITupleIterator {
private static final Logger logger = LoggerFactory.getLogger(SequentialCubeTupleIterator.class);
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/translate/DerivedFilterTranslator.java b/core-storage/src/main/java/org/apache/kylin/storage/translate/DerivedFilterTranslator.java
index 9bfdd76..8b0ddc9 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/translate/DerivedFilterTranslator.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/translate/DerivedFilterTranslator.java
@@ -27,13 +27,11 @@
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
import org.apache.kylin.cube.model.CubeDesc.DeriveType;
-import org.apache.kylin.dict.lookup.ILookupTable;
import org.apache.kylin.metadata.datatype.DataType;
import org.apache.kylin.metadata.datatype.DataTypeOrder;
import org.apache.kylin.metadata.filter.ColumnTupleFilter;
import org.apache.kylin.metadata.filter.CompareTupleFilter;
import org.apache.kylin.metadata.filter.ConstantTupleFilter;
-import org.apache.kylin.metadata.filter.FilterCodeSystemFactory;
import org.apache.kylin.metadata.filter.LogicalTupleFilter;
import org.apache.kylin.metadata.filter.TupleFilter;
import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
@@ -52,7 +50,7 @@
private static final Logger logger = LoggerFactory.getLogger(DerivedFilterTranslator.class);
- public static Pair<TupleFilter, Boolean> translate(ILookupTable lookup, DeriveInfo hostInfo, CompareTupleFilter compf) {
+ public static Pair<TupleFilter, Boolean> translate(DeriveInfo hostInfo, CompareTupleFilter compf) {
TblColRef derivedCol = compf.getColumn();
TblColRef[] hostCols = hostInfo.columns;
@@ -91,12 +89,12 @@
Set<Array<String>> satisfyingHostRecords = Sets.newHashSet();
SingleColumnTuple tuple = new SingleColumnTuple(derivedCol);
- for (String[] row : lookup) {
- tuple.value = row[di];
- if (compf.evaluate(tuple, FilterCodeSystemFactory.getFilterCodeSystem(derivedCol.getColumnDesc().getType()))) {
- collect(row, pi, satisfyingHostRecords);
- }
- }
+// for (String[] row : lookup) {
+// tuple.value = row[di];
+// if (compf.evaluate(tuple, FilterCodeSystemFactory.getFilterCodeSystem(derivedCol.getColumnDesc().getType()))) {
+// collect(row, pi, satisfyingHostRecords);
+// }
+// }
for (Array<String> entry : satisfyingHostRecords) {
for (int i = 0; i < pkCols.length; i++) {
diff --git a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
index 574bb9f..d13082b 100644
--- a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
+++ b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
@@ -21,7 +21,7 @@
import static org.junit.Assert.assertEquals;
import java.io.IOException;
-import java.math.BigDecimal;
+//import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.BitSet;
@@ -29,19 +29,19 @@
import org.apache.kylin.common.util.ByteArray;
import org.apache.kylin.common.util.BytesSerializer;
-import org.apache.kylin.common.util.Dictionary;
+//import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.common.util.ImmutableBitSet;
import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.gridtable.CubeCodeSystem;
-import org.apache.kylin.dict.NumberDictionaryForestBuilder;
-import org.apache.kylin.dict.StringBytesConverter;
-import org.apache.kylin.dict.TrieDictionaryBuilder;
-import org.apache.kylin.dimension.DictionaryDimEnc;
-import org.apache.kylin.dimension.DimensionEncoding;
-import org.apache.kylin.gridtable.GTBuilder;
+//import org.apache.kylin.cube.gridtable.CubeCodeSystem;
+//import org.apache.kylin.dict.NumberDictionaryForestBuilder;
+//import org.apache.kylin.dict.StringBytesConverter;
+////import org.apache.kylin.dict.TrieDictionaryBuilder;
+//import org.apache.kylin.dimension.DictionaryDimEnc;
+//import org.apache.kylin.dimension.DimensionEncoding;
+//import org.apache.kylin.gridtable.GTBuilder;
import org.apache.kylin.gridtable.GTFilterScanner.FilterResultCache;
import org.apache.kylin.gridtable.GTInfo;
-import org.apache.kylin.gridtable.GTInfo.Builder;
+//import org.apache.kylin.gridtable.GTInfo.Builder;
import org.apache.kylin.gridtable.GTRecord;
import org.apache.kylin.gridtable.GTScanRange;
import org.apache.kylin.gridtable.GTScanRequest;
@@ -49,8 +49,8 @@
import org.apache.kylin.gridtable.GTUtil;
import org.apache.kylin.gridtable.GridTable;
import org.apache.kylin.gridtable.IGTScanner;
-import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
-import org.apache.kylin.metadata.datatype.DataType;
+//import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
+//import org.apache.kylin.metadata.datatype.DataType;
import org.apache.kylin.metadata.filter.ColumnTupleFilter;
import org.apache.kylin.metadata.filter.CompareTupleFilter;
import org.apache.kylin.metadata.filter.ConstantTupleFilter;
@@ -65,9 +65,11 @@
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
+import org.junit.Ignore;
import org.apache.kylin.shaded.com.google.common.collect.Lists;
+@Ignore
public class DictGridTableTest extends LocalFileMetadataTestCase {
private GridTable table;
@@ -96,7 +98,7 @@
this.createTestMetadata();
- table = newTestTable();
+ table = null;
info = table.getInfo();
timeComp0 = compare(info.colRef(0), FilterOperatorEnum.LT, enc(info, 0, "2015-01-14"));
@@ -607,127 +609,127 @@
return result;
}
- public static GridTable newTestTable() throws IOException {
- GTInfo info = newInfo();
- GTSimpleMemStore store = new GTSimpleMemStore(info);
- GridTable table = new GridTable(info, store);
+// public static GridTable newTestTable() throws IOException {
+// GTInfo info = newInfo();
+// GTSimpleMemStore store = new GTSimpleMemStore(info);
+// GridTable table = new GridTable(info, store);
+//
+// GTRecord r = new GTRecord(table.getInfo());
+// GTBuilder builder = table.rebuild();
+//
+// builder.write(r.setValues("2015-01-14", "30", "Yang", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-14", "30", "Luke", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-15", "20", "Dong", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-15", "20", "Jason", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-15", "30", "Xu", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-16", "20", "Mahone", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-16", "20", "Qianhao", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-16", "30", "George", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new Long(10), new BigDecimal("10.5")));
+// builder.write(r.setValues("2015-01-17", "10", "Kejia", new Long(10), new BigDecimal("10.5")));
+// builder.close();
+//
+// return table;
+// }
- GTRecord r = new GTRecord(table.getInfo());
- GTBuilder builder = table.rebuild();
+// static GridTable newTestPerfTable() throws IOException {
+// GTInfo info = newInfo();
+// GTSimpleMemStore store = new GTSimpleMemStore(info);
+// GridTable table = new GridTable(info, store);
+//
+// GTRecord r = new GTRecord(table.getInfo());
+// GTBuilder builder = table.rebuild();
+//
+// for (int i = 0; i < 100000; i++) {
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-14", "30", "Yang", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-14", "30", "Luke", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-15", "20", "Dong", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-15", "20", "Jason", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-15", "30", "Xu", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-16", "20", "Mahone", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-16", "20", "Qianhao", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-16", "30", "George", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new Long(10), new BigDecimal("10.5")));
+//
+// for (int j = 0; j < 10; j++)
+// builder.write(r.setValues("2015-01-17", "10", "Kejia", new Long(10), new BigDecimal("10.5")));
+// }
+// builder.close();
+//
+// return table;
+// }
- builder.write(r.setValues("2015-01-14", "30", "Yang", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-14", "30", "Luke", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-15", "20", "Dong", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-15", "20", "Jason", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-15", "30", "Xu", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-16", "20", "Mahone", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-16", "20", "Qianhao", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-16", "30", "George", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new Long(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-17", "10", "Kejia", new Long(10), new BigDecimal("10.5")));
- builder.close();
+// static GTInfo newInfo() {
+// Builder builder = GTInfo.builder();
+// builder.setCodeSystem(newDictCodeSystem());
+// builder.setColumns(//
+// DataType.getType("timestamp"), //
+// DataType.getType("integer"), //
+// DataType.getType("varchar(10)"), //
+// DataType.getType("bigint"), //
+// DataType.getType("decimal") //
+// );
+// builder.setPrimaryKey(setOf(0, 1));
+// builder.setColumnPreferIndex(setOf(0));
+// builder.enableColumnBlock(new ImmutableBitSet[] { setOf(0, 1), setOf(2), setOf(3, 4) });
+// builder.enableRowBlock(4);
+// GTInfo info = builder.build();
+// return info;
+// }
- return table;
- }
+// private static CubeCodeSystem newDictCodeSystem() {
+// DimensionEncoding[] dimEncs = new DimensionEncoding[3];
+// dimEncs[1] = new DictionaryDimEnc(newDictionaryOfInteger());
+// dimEncs[2] = new DictionaryDimEnc(newDictionaryOfString());
+// return new CubeCodeSystem(dimEncs);
+// }
- static GridTable newTestPerfTable() throws IOException {
- GTInfo info = newInfo();
- GTSimpleMemStore store = new GTSimpleMemStore(info);
- GridTable table = new GridTable(info, store);
-
- GTRecord r = new GTRecord(table.getInfo());
- GTBuilder builder = table.rebuild();
-
- for (int i = 0; i < 100000; i++) {
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-14", "30", "Yang", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-14", "30", "Luke", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-15", "20", "Dong", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-15", "20", "Jason", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-15", "30", "Xu", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-16", "20", "Mahone", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-16", "20", "Qianhao", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-16", "30", "George", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new Long(10), new BigDecimal("10.5")));
-
- for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-17", "10", "Kejia", new Long(10), new BigDecimal("10.5")));
- }
- builder.close();
-
- return table;
- }
-
- static GTInfo newInfo() {
- Builder builder = GTInfo.builder();
- builder.setCodeSystem(newDictCodeSystem());
- builder.setColumns(//
- DataType.getType("timestamp"), //
- DataType.getType("integer"), //
- DataType.getType("varchar(10)"), //
- DataType.getType("bigint"), //
- DataType.getType("decimal") //
- );
- builder.setPrimaryKey(setOf(0, 1));
- builder.setColumnPreferIndex(setOf(0));
- builder.enableColumnBlock(new ImmutableBitSet[] { setOf(0, 1), setOf(2), setOf(3, 4) });
- builder.enableRowBlock(4);
- GTInfo info = builder.build();
- return info;
- }
-
- private static CubeCodeSystem newDictCodeSystem() {
- DimensionEncoding[] dimEncs = new DimensionEncoding[3];
- dimEncs[1] = new DictionaryDimEnc(newDictionaryOfInteger());
- dimEncs[2] = new DictionaryDimEnc(newDictionaryOfString());
- return new CubeCodeSystem(dimEncs);
- }
-
- private static Dictionary newDictionaryOfString() {
- TrieDictionaryBuilder<String> builder = new TrieDictionaryBuilder<>(new StringBytesConverter());
- builder.addValue("Dong");
- builder.addValue("George");
- builder.addValue("Jason");
- builder.addValue("Kejia");
- builder.addValue("Luke");
- builder.addValue("Mahone");
- builder.addValue("Qianhao");
- builder.addValue("Shaofeng");
- builder.addValue("Xu");
- builder.addValue("Yang");
- return builder.build(0);
- }
-
- private static Dictionary newDictionaryOfInteger() {
- NumberDictionaryForestBuilder builder = new NumberDictionaryForestBuilder();
- builder.addValue("10");
- builder.addValue("20");
- builder.addValue("30");
- builder.addValue("40");
- builder.addValue("50");
- builder.addValue("60");
- builder.addValue("70");
- builder.addValue("80");
- builder.addValue("90");
- builder.addValue("100");
- return builder.build();
- }
+// private static Dictionary newDictionaryOfString() {
+// TrieDictionaryBuilder<String> builder = new TrieDictionaryBuilder<>(new StringBytesConverter());
+// builder.addValue("Dong");
+// builder.addValue("George");
+// builder.addValue("Jason");
+// builder.addValue("Kejia");
+// builder.addValue("Luke");
+// builder.addValue("Mahone");
+// builder.addValue("Qianhao");
+// builder.addValue("Shaofeng");
+// builder.addValue("Xu");
+// builder.addValue("Yang");
+// return builder.build(0);
+// }
+//
+// private static Dictionary newDictionaryOfInteger() {
+// NumberDictionaryForestBuilder builder = new NumberDictionaryForestBuilder();
+// builder.addValue("10");
+// builder.addValue("20");
+// builder.addValue("30");
+// builder.addValue("40");
+// builder.addValue("50");
+// builder.addValue("60");
+// builder.addValue("70");
+// builder.addValue("80");
+// builder.addValue("90");
+// builder.addValue("100");
+// return builder.build();
+// }
public static ImmutableBitSet setOf(int... values) {
BitSet set = new BitSet();
diff --git a/datasource-sdk/pom.xml b/datasource-sdk/pom.xml
deleted file mode 100644
index 68fe878..0000000
--- a/datasource-sdk/pom.xml
+++ /dev/null
@@ -1,159 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one
- or more contributor license agreements. See the NOTICE file
- distributed with this work for additional information
- regarding copyright ownership. The ASF licenses this file
- to you under the Apache License, Version 2.0 (the
- "License"); you may not use this file except in compliance
- with the License. You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
-
- <artifactId>kylin-datasource-sdk</artifactId>
- <name>Apache Kylin - DataSource SDK</name>
-
- <parent>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin</artifactId>
- <version>4.0.0-SNAPSHOT</version>
- </parent>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-core-metadata</artifactId>
- </dependency>
-
- <dependency>
- <groupId>com.fasterxml.jackson.dataformat</groupId>
- <artifactId>jackson-dataformat-xml</artifactId>
- </dependency>
-
- <dependency>
- <groupId>org.apache.calcite</groupId>
- <artifactId>calcite-linq4j</artifactId>
- </dependency>
-
- <!--Uncomment to debug SQLServer-->
- <!--<dependency>-->
- <!--<groupId>com.microsoft.sqlserver</groupId>-->
- <!--<artifactId>mssql-jdbc</artifactId>-->
- <!--<version>${mssql-jdbc.version}</version>-->
- <!--<scope>provided</scope>-->
- <!--</dependency>-->
-
- <!--Test and Environment-->
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-core-common</artifactId>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-core-metadata</artifactId>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>com.h2database</groupId>
- <artifactId>h2</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.postgresql</groupId>
- <artifactId>postgresql</artifactId>
- <version>42.1.1</version>
- <scope>provided</scope>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <executions>
- <execution>
- <id>0</id>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <minimizeJar>false</minimizeJar>
- <shadedArtifactAttached>true</shadedArtifactAttached>
- <shadedClassifierName>lib</shadedClassifierName>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <includes>
- <include>org/apache/kylin/sdk/datasource/adaptor/**</include>
- <include>datasource/**</include>
- </includes>
- </filter>
- </filters>
- </configuration>
- </execution>
- <execution>
- <id>1</id>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <minimizeJar>false</minimizeJar>
- <shadedArtifactAttached>true</shadedArtifactAttached>
- <shadedClassifierName>framework</shadedClassifierName>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <includes>
- <include>org/apache/kylin/sdk/datasource/framework/**</include>
- </includes>
- </filter>
- </filters>
- </configuration>
- </execution>
- <execution>
- <id>2</id>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <minimizeJar>false</minimizeJar>
- <shadedArtifactAttached>true</shadedArtifactAttached>
- <shadedClassifierName>testsuite</shadedClassifierName>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <includes>
- <include>org/apache/kylin/sdk/datasource/framework/**</include>
- <include>org/apache/kylin/sdk/datasource/adaptor/**</include>
- </includes>
- </filter>
- </filters>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-</project>
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 75d33a9..d8f73be 100644
--- a/pom.xml
+++ b/pom.xml
@@ -284,11 +284,11 @@
<artifactId>kylin-core-metadata</artifactId>
<version>${project.version}</version>
</dependency>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-core-dictionary</artifactId>
- <version>${project.version}</version>
- </dependency>
+<!-- <dependency>-->
+<!-- <groupId>org.apache.kylin</groupId>-->
+<!-- <artifactId>kylin-core-dictionary</artifactId>-->
+<!-- <version>${project.version}</version>-->
+<!-- </dependency>-->
<dependency>
<groupId>org.apache.kylin</groupId>
<artifactId>kylin-core-cube</artifactId>
@@ -463,17 +463,17 @@
<!-- <version>${project.version}</version>-->
<!-- </dependency>-->
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-datasource-sdk</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-datasource-sdk</artifactId>
- <version>${project.version}</version>
- <classifier>framework</classifier>
- </dependency>
+<!-- <dependency>-->
+<!-- <groupId>org.apache.kylin</groupId>-->
+<!-- <artifactId>kylin-datasource-sdk</artifactId>-->
+<!-- <version>${project.version}</version>-->
+<!-- </dependency>-->
+<!-- <dependency>-->
+<!-- <groupId>org.apache.kylin</groupId>-->
+<!-- <artifactId>kylin-datasource-sdk</artifactId>-->
+<!-- <version>${project.version}</version>-->
+<!-- <classifier>framework</classifier>-->
+<!-- </dependency>-->
<!-- <dependency>-->
<!-- <groupId>org.apache.kylin</groupId>-->
<!-- <artifactId>kylin-assembly</artifactId>-->
@@ -1530,7 +1530,7 @@
<module>external</module>
<module>core-common</module>
<module>core-metadata</module>
- <module>core-dictionary</module>
+<!-- <module>core-dictionary</module>-->
<module>core-cube</module>
<module>core-job</module>
<module>core-storage</module>
@@ -1552,7 +1552,7 @@
<module>metrics-reporter-hive</module>
<module>metrics-reporter-kafka</module>
<module>cache</module>
- <module>datasource-sdk</module>
+<!-- <module>datasource-sdk</module>-->
<!-- <module>storage-stream</module>-->
<!-- <module>stream-receiver</module>-->
<!-- <module>stream-coordinator</module>-->
@@ -1745,114 +1745,6 @@
</plugins>
</build>
</profile>
- <profile>
- <id>cdh5.7</id>
- <properties>
- <hadoop2.version>2.6.0-cdh5.7.0</hadoop2.version>
- <yarn.version>2.6.0-cdh5.7.0</yarn.version>
- <hive.version>1.1.0-cdh5.7.0</hive.version>
- <hive-hcatalog.version>1.1.0-cdh5.7.0</hive-hcatalog.version>
- <hbase-hadoop2.version>1.2.0-cdh5.7.0</hbase-hadoop2.version>
- <zookeeper.version>3.4.5-cdh5.7.0</zookeeper.version>
- </properties>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-compiler-plugin</artifactId>
- <configuration>
- <fork>true</fork>
- <meminitial>1024m</meminitial>
- <maxmem>2048m</maxmem>
- </configuration>
- </plugin>
-
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
- <executions>
- <execution>
- <id>copy-jamm</id>
- <goals>
- <goal>copy</goal>
- </goals>
- <phase>generate-test-resources</phase>
- <configuration>
- <artifactItems>
- <artifactItem>
- <groupId>com.github.jbellis</groupId>
- <artifactId>jamm</artifactId>
- <outputDirectory>${project.build.testOutputDirectory}
- </outputDirectory>
- <destFileName>jamm.jar</destFileName>
- </artifactItem>
- </artifactItems>
- </configuration>
- </execution>
- </executions>
- </plugin>
-
- <plugin>
- <groupId>org.jacoco</groupId>
- <artifactId>jacoco-maven-plugin</artifactId>
- <configuration>
- <append>true</append>
- <destFile>
- ${sonar.jacoco.reportPaths}
- </destFile>
- </configuration>
- <executions>
- <execution>
- <id>pre-test</id>
- <goals>
- <goal>prepare-agent</goal>
- </goals>
- <configuration>
- <propertyName>surefireArgLine</propertyName>
- </configuration>
- </execution>
- <execution>
- <id>post-test</id>
- <phase>test</phase>
- <goals>
- <goal>report</goal>
- </goals>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-surefire-plugin</artifactId>
- <version>2.21.0</version>
- <configuration>
- <reportsDirectory>${project.basedir}/../target/surefire-reports
- </reportsDirectory>
- <excludes>
- <exclude>**/IT*.java</exclude>
- <exclude>org.apache.kylin.engine.spark2.NManualBuildAndQueryCuboidTest</exclude>
- <exclude>org.apache.kylin.engine.spark2.NBuildAndQueryTest</exclude>
- <exclude>org.apache.kylin.engine.spark2.NBadQueryAndPushDownTest</exclude>
- </excludes>
- <systemProperties>
- <property>
- <name>buildCubeUsingProvidedData</name>
- <value>false</value>
- </property>
- <property>
- <name>log4j.configuration</name>
- <value>
- file:${project.basedir}/../build/conf/kylin-tools-log4j.properties
- </value>
- </property>
- </systemProperties>
- <argLine>-javaagent:${project.build.testOutputDirectory}/jamm.jar
- ${argLine} ${surefireArgLine}
- </argLine>
- </configuration>
- </plugin>
- </plugins>
-</build>
-</profile>
<profile>
<!-- This profile adds/overrides few features of the 'apache-release'
profile in the parent pom. -->
@@ -2143,6 +2035,103 @@
<spark.version.dir>spark31</spark.version.dir>
<jackson.version>2.10.0</jackson.version>
</properties>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <configuration>
+ <fork>true</fork>
+ <meminitial>1024m</meminitial>
+ <maxmem>2048m</maxmem>
+ </configuration>
+ </plugin>
+
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-dependency-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>copy-jamm</id>
+ <goals>
+ <goal>copy</goal>
+ </goals>
+ <phase>generate-test-resources</phase>
+ <configuration>
+ <artifactItems>
+ <artifactItem>
+ <groupId>com.github.jbellis</groupId>
+ <artifactId>jamm</artifactId>
+ <outputDirectory>${project.build.testOutputDirectory}
+ </outputDirectory>
+ <destFileName>jamm.jar</destFileName>
+ </artifactItem>
+ </artifactItems>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+
+ <plugin>
+ <groupId>org.jacoco</groupId>
+ <artifactId>jacoco-maven-plugin</artifactId>
+ <configuration>
+ <append>true</append>
+ <destFile>
+ ${sonar.jacoco.reportPaths}
+ </destFile>
+ </configuration>
+ <executions>
+ <execution>
+ <id>pre-test</id>
+ <goals>
+ <goal>prepare-agent</goal>
+ </goals>
+ <configuration>
+ <propertyName>surefireArgLine</propertyName>
+ </configuration>
+ </execution>
+ <execution>
+ <id>post-test</id>
+ <phase>test</phase>
+ <goals>
+ <goal>report</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.21.0</version>
+ <configuration>
+ <reportsDirectory>${project.basedir}/../target/surefire-reports
+ </reportsDirectory>
+ <excludes>
+ <exclude>**/IT*.java</exclude>
+ <exclude>org.apache.kylin.engine.spark2.NManualBuildAndQueryCuboidTest</exclude>
+ <exclude>org.apache.kylin.engine.spark2.NBuildAndQueryTest</exclude>
+ <exclude>org.apache.kylin.engine.spark2.NBadQueryAndPushDownTest</exclude>
+ </excludes>
+ <systemProperties>
+ <property>
+ <name>buildCubeUsingProvidedData</name>
+ <value>false</value>
+ </property>
+ <property>
+ <name>log4j.configuration</name>
+ <value>
+ file:${project.basedir}/../build/conf/kylin-tools-log4j.properties
+ </value>
+ </property>
+ </systemProperties>
+ <argLine>-javaagent:${project.build.testOutputDirectory}/jamm.jar
+ ${argLine} ${surefireArgLine}
+ </argLine>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
<!--
<activation>
<activeByDefault>true</activeByDefault>
diff --git a/query/pom.xml b/query/pom.xml
index 4ee56d3..87f6f49 100644
--- a/query/pom.xml
+++ b/query/pom.xml
@@ -58,10 +58,10 @@
<groupId>org.apache.kylin</groupId>
<artifactId>kylin-core-storage</artifactId>
</dependency>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-datasource-sdk</artifactId>
- </dependency>
+<!-- <dependency>-->
+<!-- <groupId>org.apache.kylin</groupId>-->
+<!-- <artifactId>kylin-datasource-sdk</artifactId>-->
+<!-- </dependency>-->
<!-- Provided -->
<dependency>
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/DictionaryEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/DictionaryEnumerator.java
index 6af65ee..af02cdf 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/DictionaryEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/DictionaryEnumerator.java
@@ -20,13 +20,11 @@
import java.util.Iterator;
import java.util.List;
-import java.util.Set;
import org.apache.calcite.linq4j.Enumerator;
+import org.apache.kylin.common.annotation.Clarification;
import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
import org.apache.kylin.metadata.model.TblColRef;
import org.apache.kylin.metadata.realization.IRealization;
import org.apache.kylin.metadata.tuple.Tuple;
@@ -37,8 +35,8 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
+@Clarification(deprecated = true, msg = "Only for HBase storage")
public class DictionaryEnumerator implements Enumerator<Object[]> {
private final static Logger logger = LoggerFactory.getLogger(DictionaryEnumerator.class);
@@ -95,21 +93,21 @@
}
public static List<Dictionary<String>> getAllDictionaries(TblColRef col, IRealization realization) {
- Set<Dictionary<String>> result = Sets.newHashSet();
- if (realization instanceof CubeInstance) {
- final CubeInstance cube = (CubeInstance) realization;
- for (CubeSegment segment : cube.getSegments(SegmentStatusEnum.READY)) {
- result.add(segment.getDictionary(col));
- }
- } else if (realization instanceof HybridInstance) {
- final HybridInstance hybridInstance = (HybridInstance) realization;
- for (IRealization entry : hybridInstance.getRealizations()) {
- result.addAll(getAllDictionaries(col, entry));
- }
- } else {
- throw new IllegalStateException("All leaf realizations should be CubeInstance");
- }
- return Lists.newArrayList(result);
+// Set<Dictionary<String>> result = Sets.newHashSet();
+// if (realization instanceof CubeInstance) {
+// final CubeInstance cube = (CubeInstance) realization;
+// for (CubeSegment segment : cube.getSegments(SegmentStatusEnum.READY)) {
+// result.add(segment.getDictionary(col));
+// }
+// } else if (realization instanceof HybridInstance) {
+// final HybridInstance hybridInstance = (HybridInstance) realization;
+// for (IRealization entry : hybridInstance.getRealizations()) {
+// result.addAll(getAllDictionaries(col, entry));
+// }
+// } else {
+// throw new IllegalStateException("All leaf realizations should be CubeInstance");
+// }
+ return Lists.newArrayList();
}
@Override
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
index ad2e20c..c9e61f8 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
@@ -18,7 +18,6 @@
package org.apache.kylin.query.enumerator;
-import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
@@ -27,12 +26,12 @@
import java.util.Set;
import org.apache.calcite.linq4j.Enumerator;
+import org.apache.kylin.common.annotation.Clarification;
import org.apache.kylin.common.debug.BackdoorToggles;
import org.apache.kylin.common.util.StringUtil;
import org.apache.kylin.cube.CubeInstance;
import org.apache.kylin.cube.CubeManager;
import org.apache.kylin.cube.model.DimensionDesc;
-import org.apache.kylin.dict.lookup.ILookupTable;
import org.apache.kylin.metadata.model.ColumnDesc;
import org.apache.kylin.metadata.project.ProjectInstance;
import org.apache.kylin.metadata.project.RealizationEntry;
@@ -44,12 +43,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-/**
- */
+@Clarification(deprecated = true, msg = "Only for HBase storage")
public class LookupTableEnumerator implements Enumerator<Object[]> {
private final static Logger logger = LoggerFactory.getLogger(LookupTableEnumerator.class);
- private final ILookupTable lookupTable;
+// private ILookupTable lookupTable;
private final List<ColumnDesc> colDescs;
private final Object[] current;
private Iterator<String[]> iterator;
@@ -97,7 +95,7 @@
throw new IllegalStateException("No dimension with derived columns found for lookup table " + lookupTableName + ", cube desc " + cube.getDescriptor());
CubeManager cubeMgr = CubeManager.getInstance(cube.getConfig());
- this.lookupTable = cubeMgr.getLookupTable(cube.getLatestReadySegment(), dim.getJoin());
+// this.lookupTable = cubeMgr.getLookupTable(cube.getLatestReadySegment(), dim.getJoin());
OLAPTable olapTable = (OLAPTable) olapContext.firstTableScan.getOlapTable();
this.colDescs = olapTable.getSourceColumns();
@@ -132,16 +130,16 @@
@Override
public void reset() {
- this.iterator = lookupTable.iterator();
+// this.iterator = lookupTable.iterator();
}
@Override
public void close() {
- try {
- lookupTable.close();
- } catch (IOException e) {
- logger.error("error when close lookup table", e);
- }
+// try {
+// lookupTable.close();
+// } catch (IOException e) {
+// logger.error("error when close lookup table", e);
+// }
}
}
diff --git a/query/src/main/java/org/apache/kylin/query/pushdown/PushdownRunnerSDKImpl.java b/query/src/main/java/org/apache/kylin/query/pushdown/PushdownRunnerSDKImpl.java
index 3ead596..2a733f5 100644
--- a/query/src/main/java/org/apache/kylin/query/pushdown/PushdownRunnerSDKImpl.java
+++ b/query/src/main/java/org/apache/kylin/query/pushdown/PushdownRunnerSDKImpl.java
@@ -17,95 +17,10 @@
*/
package org.apache.kylin.query.pushdown;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.LinkedList;
-import java.util.List;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.DBUtils;
-import org.apache.kylin.metadata.querymeta.SelectedColumnMeta;
-import org.apache.kylin.sdk.datasource.framework.JdbcConnector;
-import org.apache.kylin.sdk.datasource.framework.SourceConnectorFactory;
-import org.apache.kylin.source.adhocquery.AbstractPushdownRunner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.kylin.common.annotation.Clarification;
-public class PushdownRunnerSDKImpl extends AbstractPushdownRunner {
- public static final Logger logger = LoggerFactory.getLogger(PushdownRunnerSDKImpl.class);
+@Clarification(deprecated = true)
+public class PushdownRunnerSDKImpl {
- private JdbcConnector dataSource;
-
- @Override
- public void init(KylinConfig config) {
- dataSource = SourceConnectorFactory.getJdbcConnector(config);
- }
-
- @Override
- public void executeQuery(String sql, List<List<String>> results, List<SelectedColumnMeta> columnMetas) {
- //extract column metadata
- ResultSet rs = null;
- ResultSetMetaData metaData;
- int columnCount;
- try (Connection conn = dataSource.getConnection(); Statement state = conn.createStatement()) {
- rs = state.executeQuery(sql);
-
- extractResults(rs, results);
- metaData = rs.getMetaData();
- columnCount = metaData.getColumnCount();
-
- // fill in selected column meta
- for (int i = 1; i <= columnCount; ++i) {
- int kylinTypeId = dataSource.toKylinTypeId(metaData.getColumnTypeName(i), metaData.getColumnType(i));
- String kylinTypeName = dataSource.toKylinTypeName(kylinTypeId);
- columnMetas.add(new SelectedColumnMeta(metaData.isAutoIncrement(i), metaData.isCaseSensitive(i), false,
- metaData.isCurrency(i), metaData.isNullable(i), false, metaData.getColumnDisplaySize(i),
- metaData.getColumnLabel(i), metaData.getColumnName(i), null, null, null,
- metaData.getPrecision(i), metaData.getScale(i), kylinTypeId, kylinTypeName,
- metaData.isReadOnly(i), false, false));
- }
- } catch (Exception e) {
- throw new RuntimeException("executeQuery failed", e);
- } finally {
- DBUtils.closeQuietly(rs);
- }
- }
-
- @Override
- public void executeUpdate(String sql) {
- try {
- dataSource.executeUpdate(sql);
- } catch (Exception e) {
- throw new RuntimeException("executeUpdate failed", e);
- }
- }
-
- @Override
- public String convertSql(KylinConfig kylinConfig, String sql, String project, String defaultSchema,
- boolean isPrepare) {
- String converted = sql;
-
- // SDK convert
- String ret = dataSource.convertSql(converted);
- if (!converted.equals(ret)) {
- logger.debug("the query is converted to {} after applying SDK converter.", ret);
- }
- return ret;
- }
-
- private void extractResults(ResultSet resultSet, List<List<String>> results) throws SQLException {
- List<String> oneRow = new LinkedList<>();
-
- while (resultSet.next()) {
- for (int i = 0; i < resultSet.getMetaData().getColumnCount(); i++) {
- oneRow.add((resultSet.getString(i + 1)));
- }
-
- results.add(new LinkedList<>(oneRow));
- oneRow.clear();
- }
- }
}
diff --git a/server-base/pom.xml b/server-base/pom.xml
index e9fc1b9..55d2706 100644
--- a/server-base/pom.xml
+++ b/server-base/pom.xml
@@ -88,16 +88,16 @@
<!-- <groupId>org.apache.kylin</groupId>-->
<!-- <artifactId>kylin-source-jdbc</artifactId>-->
<!-- </dependency>-->
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-datasource-sdk</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-datasource-sdk</artifactId>
- <classifier>framework</classifier>
- </dependency>
+<!-- <dependency>-->
+<!-- <groupId>org.apache.kylin</groupId>-->
+<!-- <artifactId>kylin-datasource-sdk</artifactId>-->
+<!-- <scope>provided</scope>-->
+<!-- </dependency>-->
+<!-- <dependency>-->
+<!-- <groupId>org.apache.kylin</groupId>-->
+<!-- <artifactId>kylin-datasource-sdk</artifactId>-->
+<!-- <classifier>framework</classifier>-->
+<!-- </dependency>-->
<!-- <dependency>-->
<!-- <groupId>org.apache.kylin</groupId>-->
<!-- <artifactId>kylin-stream-coordinator</artifactId>-->
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index 00370e3..23ad6a7 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -36,7 +36,6 @@
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.common.util.RandomUtil;
import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.cube.cuboid.CuboidScheduler;
import org.apache.kylin.cube.cuboid.TreeCuboidScheduler;
@@ -101,7 +100,6 @@
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonMappingException;
import org.apache.kylin.shaded.com.google.common.collect.Lists;
-import org.apache.kylin.shaded.com.google.common.collect.Maps;
import org.apache.kylin.shaded.com.google.common.collect.Sets;
/**
@@ -176,14 +174,7 @@
@RequestMapping(value = "validEncodings", method = { RequestMethod.GET }, produces = { "application/json" })
@ResponseBody
public Map<String, Integer> getValidEncodings() {
- Map<String, Integer> encodings;
- try {
- encodings = DimensionEncodingFactory.getValidEncodings();
- } catch (Exception e) {
- logger.error("Error when getting valid encodings", e);
- return Maps.newHashMap();
- }
- return encodings;
+ throw new UnsupportedOperationException("Unsupported in Kylin 4.0 .");
}
@RequestMapping(value = "/{cubeName}", method = { RequestMethod.GET }, produces = { "application/json" })
@@ -313,14 +304,7 @@
@ResponseBody
public CubeInstance rebuildLookupSnapshot(@PathVariable String cubeName, @PathVariable String segmentName,
@RequestParam(value = "lookupTable") String lookupTable) {
- try {
- final CubeManager cubeMgr = cubeService.getCubeManager();
- final CubeInstance cube = cubeMgr.getCube(cubeName);
- return cubeService.rebuildLookupSnapshot(cube, segmentName, lookupTable);
- } catch (IOException e) {
- logger.error(e.getLocalizedMessage(), e);
- throw new InternalErrorException(e.getLocalizedMessage(), e);
- }
+ throw new UnsupportedOperationException("Unsupported in Kylin 4.0 .");
}
/**
@@ -401,15 +385,7 @@
@RequestMapping(value = "/{cubeName}/build2", method = { RequestMethod.PUT }, produces = { "application/json" })
@ResponseBody
public JobInstance build2(@PathVariable String cubeName, @RequestBody JobBuildRequest2 req) {
- try {
- Class<?> clazz = Class.forName("org.apache.kafka.clients.consumer.KafkaConsumer");
- if (clazz == null) {
- throw new ClassNotFoundException();
- }
- } catch (ClassNotFoundException e) {
- throw new InternalErrorException("Could not find Kafka dependency");
- }
- return rebuild2(cubeName, req);
+ throw new UnsupportedOperationException("Unsupported in Kylin 4.0 .");
}
/**
@@ -418,9 +394,7 @@
@RequestMapping(value = "/{cubeName}/rebuild2", method = { RequestMethod.PUT }, produces = { "application/json" })
@ResponseBody
public JobInstance rebuild2(@PathVariable String cubeName, @RequestBody JobBuildRequest2 req) {
- return buildInternal(cubeName, null, new SegmentRange(req.getSourceOffsetStart(), req.getSourceOffsetEnd()),
- req.getSourcePartitionOffsetStart(), req.getSourcePartitionOffsetEnd(), req.getBuildType(),
- req.isForce(), req.getPriorityOffset());
+ throw new UnsupportedOperationException("Unsupported in Kylin 4.0 .");
}
private JobInstance buildInternal(String cubeName, TSRange tsRange, SegmentRange segRange, //
@@ -1079,38 +1053,19 @@
return cubeService.getCuboidHitFrequency(cubeName, false);
}
-// /**
-// * Initiate the very beginning of a streaming cube. Will seek the latest offests of each partition from streaming
-// * source (kafka) and record in the cube descriptor; In the first build job, it will use these offests as the start point.
-// *
-// * @param cubeName
-// * @return
-// */
-// @RequestMapping(value = "/{cubeName}/init_start_offsets", method = { RequestMethod.PUT }, produces = {
-// "application/json" })
-// @ResponseBody
-// public GeneralResponse initStartOffsets(@PathVariable String cubeName) {
-// checkCubeExists(cubeName);
-// CubeInstance cubeInstance = cubeService.getCubeManager().getCube(cubeName);
-// if (cubeInstance.getSourceType() != ISourceAware.ID_STREAMING) {
-// String msg = "Cube '" + cubeName + "' is not a Streaming Cube.";
-// throw new IllegalArgumentException(msg);
-// }
-//
-// final GeneralResponse response = new GeneralResponse();
-// try {
-// final Map<Integer, Long> startOffsets = KafkaClient.getLatestOffsets(cubeInstance);
-// CubeDesc desc = cubeInstance.getDescriptor();
-// desc.setPartitionOffsetStart(startOffsets);
-// cubeService.getCubeDescManager().updateCubeDesc(desc);
-// response.setProperty("result", "success");
-// response.setProperty("offsets", startOffsets.toString());
-// } catch (Throwable e) {
-// throw new RuntimeException(e);
-// }
-//
-// return response;
-// }
+ /**
+ * Initiate the very beginning of a streaming cube. Will seek the latest offests of each partition from streaming
+ * source (kafka) and record in the cube descriptor; In the first build job, it will use these offests as the start point.
+ *
+ * @param cubeName
+ * @return
+ */
+ @RequestMapping(value = "/{cubeName}/init_start_offsets", method = { RequestMethod.PUT }, produces = {
+ "application/json" })
+ @ResponseBody
+ public GeneralResponse initStartOffsets(@PathVariable String cubeName) {
+ throw new UnsupportedOperationException("Unsupported in Kylin 4.0 .");
+ }
private CubeDesc deserializeCubeDesc(CubeRequest cubeRequest) {
CubeDesc desc = null;
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index 90bb7bb..9d92304 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -224,7 +224,7 @@
@ResponseBody
public List<TableSnapshotResponse> getTableSnapshots(@PathVariable final String project,
@PathVariable final String tableName) throws IOException {
- return tableService.getLookupTableSnapshots(project, tableName);
+ throw new UnsupportedOperationException("API getTableSnapshots is not supported in Kylin 4.0 .");
}
@RequestMapping(value = "/supported_datetime_patterns", method = { RequestMethod.GET })
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index bd30109..6fcdc75 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -64,7 +64,7 @@
import org.apache.kylin.metadata.model.MeasureDesc;
import org.apache.kylin.metadata.model.SegmentRange;
import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.apache.kylin.metadata.model.TableDesc;
+//import org.apache.kylin.metadata.model.TableDesc;
import org.apache.kylin.metadata.project.ProjectInstance;
import org.apache.kylin.metadata.project.ProjectManager;
import org.apache.kylin.metadata.project.RealizationEntry;
@@ -592,21 +592,6 @@
return getCubeManager().updateCube(update);
}
- public CubeInstance rebuildLookupSnapshot(CubeInstance cube, String segmentName, String lookupTable)
- throws IOException {
- aclEvaluate.checkProjectOperationPermission(cube);
- Message msg = MsgPicker.getMsg();
- TableDesc tableDesc = getTableManager().getTableDesc(lookupTable, cube.getProject());
- if (tableDesc.isView()) {
- throw new BadRequestException(
- String.format(Locale.ROOT, msg.getREBUILD_SNAPSHOT_OF_VIEW(), tableDesc.getName()));
- }
- CubeSegment seg = cube.getSegment(segmentName, SegmentStatusEnum.READY);
- getCubeManager().buildSnapshotTable(seg, lookupTable, null);
-
- return cube;
- }
-
public CubeInstance deleteSegmentById(CubeInstance cube, String uuid) throws IOException {
aclEvaluate.checkProjectWritePermission(cube);
Message msg = MsgPicker.getMsg();
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
index 6f282ff..10ab224 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/TableService.java
@@ -23,7 +23,6 @@
import java.io.InputStreamReader;
import java.nio.charset.Charset;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
@@ -44,12 +43,7 @@
import org.apache.kylin.common.util.JsonUtil;
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.common.util.RandomUtil;
-import org.apache.kylin.cube.CubeInstance;
import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.dict.lookup.SnapshotManager;
-import org.apache.kylin.dict.lookup.SnapshotTable;
-import org.apache.kylin.engine.spark.source.CsvSource;
import org.apache.kylin.metadata.TableMetadataManager;
import org.apache.kylin.metadata.model.ColumnDesc;
import org.apache.kylin.metadata.model.CsvColumnDesc;
@@ -61,10 +55,7 @@
import org.apache.kylin.rest.msg.Message;
import org.apache.kylin.rest.msg.MsgPicker;
import org.apache.kylin.rest.response.TableDescResponse;
-import org.apache.kylin.rest.response.TableSnapshotResponse;
import org.apache.kylin.rest.util.AclEvaluate;
-import org.apache.kylin.source.IReadableTable;
-import org.apache.kylin.source.IReadableTable.TableSignature;
import org.apache.kylin.source.ISource;
import org.apache.kylin.source.ISourceMetadataExplorer;
import org.apache.kylin.source.SourceManager;
@@ -80,7 +71,6 @@
import org.apache.kylin.shaded.com.google.common.collect.Iterables;
import org.apache.kylin.shaded.com.google.common.collect.LinkedHashMultimap;
import org.apache.kylin.shaded.com.google.common.collect.Lists;
-import org.apache.kylin.shaded.com.google.common.collect.Maps;
import org.apache.kylin.shaded.com.google.common.collect.SetMultimap;
@Component("tableService")
@@ -340,149 +330,9 @@
TableDescResponse rtableDesc = cloneTableDesc(table, prj);
descs.add(rtableDesc);
}
-
return descs;
}
-// public void calculateCardinalityIfNotPresent(String[] tables, String submitter, String prj) throws Exception {
-// // calculate cardinality for Hive source
-// ProjectInstance projectInstance = getProjectManager().getProject(prj);
-// if (projectInstance == null || projectInstance.getSourceType() != ISourceAware.ID_HIVE) {
-// return;
-// }
-// TableMetadataManager metaMgr = getTableManager();
-// ExecutableManager exeMgt = ExecutableManager.getInstance(getConfig());
-// for (String table : tables) {
-// TableExtDesc tableExtDesc = metaMgr.getTableExt(table, prj);
-// String jobID = tableExtDesc.getJodID();
-// if (null == jobID || ExecutableState.RUNNING != exeMgt.getOutput(jobID).getState()) {
-// calculateCardinality(table, submitter, prj);
-// }
-// }
-// }
-
- public List<TableSnapshotResponse> getLookupTableSnapshots(String project, String tableName) throws IOException {
- TableDesc tableDesc = getTableManager().getTableDesc(tableName, project);
- if (SourceManager.getSource(tableDesc).getClass() == CsvSource.class) {
- return new ArrayList<>();
- }
- IReadableTable hiveTable = SourceManager.createReadableTable(tableDesc, null);
- TableSignature signature = hiveTable.getSignature();
- return internalGetLookupTableSnapshots(tableName, signature);
- }
-
- List<TableSnapshotResponse> internalGetLookupTableSnapshots(String tableName, TableSignature signature)
- throws IOException {
- SnapshotManager snapshotManager = SnapshotManager.getInstance(getConfig());
- List<SnapshotTable> metaStoreTableSnapshots = snapshotManager.getSnapshots(tableName, signature);
-
- Map<String, List<String>> snapshotUsageMap = getSnapshotUsages();
-
- List<TableSnapshotResponse> result = Lists.newArrayList();
-
- for (SnapshotTable metaStoreTableSnapshot : metaStoreTableSnapshots) {
- TableSnapshotResponse response = new TableSnapshotResponse();
- response.setSnapshotID(metaStoreTableSnapshot.getId());
- response.setSnapshotType(TableSnapshotResponse.TYPE_INNER);
- response.setLastBuildTime(metaStoreTableSnapshot.getLastBuildTime());
- response.setStorageType(SnapshotTable.STORAGE_TYPE_METASTORE);
- response.setSourceTableSize(metaStoreTableSnapshot.getSignature().getSize());
- response.setSourceTableLastModifyTime(metaStoreTableSnapshot.getSignature().getLastModifiedTime());
- response.setCubesAndSegmentsUsage(snapshotUsageMap.get(metaStoreTableSnapshot.getResourcePath()));
- result.add(response);
- }
-
- return result;
- }
-
- /**
- * @return Map of SnapshotID, CubeName or SegmentName list
- */
- private Map<String, List<String>> getSnapshotUsages() {
- CubeManager cubeManager = CubeManager.getInstance(getConfig());
- Map<String, List<String>> snapshotCubeSegmentMap = Maps.newHashMap();
- for (CubeInstance cube : cubeManager.listAllCubes()) {
- Collection<String> cubeSnapshots = cube.getSnapshots().values();
- for (String cubeSnapshot : cubeSnapshots) {
- List<String> usages = snapshotCubeSegmentMap.get(cubeSnapshot);
- if (usages == null) {
- usages = Lists.newArrayList();
- snapshotCubeSegmentMap.put(cubeSnapshot, usages);
- }
- usages.add(cube.getName());
- }
- for (CubeSegment segment : cube.getSegments()) {
- Collection<String> segmentSnapshots = segment.getSnapshotPaths();
- for (String segmentSnapshot : segmentSnapshots) {
- List<String> usages = snapshotCubeSegmentMap.get(segmentSnapshot);
- if (usages == null) {
- usages = Lists.newArrayList();
- snapshotCubeSegmentMap.put(segmentSnapshot, usages);
- }
- usages.add(cube.getName() + ":" + segment.getName());
- }
- }
- }
- return snapshotCubeSegmentMap;
- }
-
-// /**
-// * Generate cardinality for table This will trigger a hadoop job
-// * The result will be merged into table exd info
-// *
-// * @param tableName
-// */
-// public void calculateCardinality(String tableName, String submitter, String prj) throws Exception {
-// aclEvaluate.checkProjectWritePermission(prj);
-// Message msg = MsgPicker.getMsg();
-//
-// tableName = normalizeHiveTableName(tableName);
-// TableDesc table = getTableManager().getTableDesc(tableName, prj);
-// final TableExtDesc tableExt = getTableManager().getTableExt(tableName, prj);
-// if (table == null) {
-// BadRequestException e = new BadRequestException(
-// String.format(Locale.ROOT, msg.getTABLE_DESC_NOT_FOUND(), tableName));
-// logger.error("Cannot find table descriptor " + tableName, e);
-// throw e;
-// }
-//
-// CardinalityExecutable job = new CardinalityExecutable();
-// //make sure the job could be scheduled when the DistributedScheduler is enable.
-// job.setParam("segmentId", tableName);
-// job.setName("Hive Column Cardinality calculation for table '" + tableName + "'");
-// job.setSubmitter(submitter);
-//
-// String outPath = getConfig().getHdfsWorkingDirectory() + "cardinality/" + job.getId() + "/" + tableName;
-// String param = "-table " + tableName + " -output " + outPath + " -project " + prj;
-//
-// if (getConfig().isSparkCardinalityEnabled()) { // use spark engine to calculate cardinality
-// SparkExecutable step1 = new SparkExecutable();
-// step1.setClassName(SparkColumnCardinality.class.getName());
-// step1.setParam(SparkColumnCardinality.OPTION_OUTPUT.getOpt(), outPath);
-// step1.setParam(SparkColumnCardinality.OPTION_PRJ.getOpt(), prj);
-// step1.setParam(SparkColumnCardinality.OPTION_TABLE_NAME.getOpt(), tableName);
-// step1.setParam(SparkColumnCardinality.OPTION_COLUMN_COUNT.getOpt(), String.valueOf(table.getColumnCount()));
-// job.addTask(step1);
-// } else {
-// MapReduceExecutable step1 = new MapReduceExecutable();
-// step1.setMapReduceJobClass(HiveColumnCardinalityJob.class);
-// step1.setMapReduceParams(param);
-// step1.setParam("segmentId", tableName);
-// job.addTask(step1);
-// }
-//
-// HadoopShellExecutable step2 = new HadoopShellExecutable();
-//
-// step2.setJobClass(HiveColumnCardinalityUpdateJob.class);
-// step2.setJobParams(param);
-// step2.setParam("segmentId", tableName);
-// job.addTask(step2);
-// tableExt.setJodID(job.getId());
-// getTableManager().saveTableExt(tableExt, prj);
-//
-// getExecutableManager().addJob(job);
-// }
-
public String normalizeHiveTableName(String tableName) {
String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
return (dbTableName[0] + "." + dbTableName[1]).toUpperCase(Locale.ROOT);
diff --git a/server-base/src/test/java/org/apache/kylin/rest/service/TableServiceTest.java b/server-base/src/test/java/org/apache/kylin/rest/service/TableServiceTest.java
index 86d34ac..b313353 100644
--- a/server-base/src/test/java/org/apache/kylin/rest/service/TableServiceTest.java
+++ b/server-base/src/test/java/org/apache/kylin/rest/service/TableServiceTest.java
@@ -18,18 +18,13 @@
package org.apache.kylin.rest.service;
-import java.io.IOException;
-import java.util.List;
import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.rest.response.TableSnapshotResponse;
-import org.apache.kylin.source.IReadableTable.TableSignature;
import org.junit.After;
-import org.junit.Assert;
import org.junit.Before;
-import org.junit.Test;
+import org.junit.Ignore;
-
+@Ignore
public class TableServiceTest extends LocalFileMetadataTestCase {
private TableService tableService;
@@ -44,10 +39,4 @@
this.cleanupTestMetadata();
}
- @Test
- public void testGetTableSnapshots() throws IOException {
- TableSignature tableSignature = new TableSignature("TEST_CAL_DT.csv", 100, System.currentTimeMillis());
- List<TableSnapshotResponse> snapshotResponseList = tableService.internalGetLookupTableSnapshots("EDW.TEST_CAL_DT", tableSignature);
- Assert.assertEquals(8, snapshotResponseList.size());
- }
}
diff --git a/server/pom.xml b/server/pom.xml
index 4c34cdc..588a93e 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -47,11 +47,11 @@
</exclusion>
</exclusions>
</dependency>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-datasource-sdk</artifactId>
- <scope>provided</scope>
- </dependency>
+<!-- <dependency>-->
+<!-- <groupId>org.apache.kylin</groupId>-->
+<!-- <artifactId>kylin-datasource-sdk</artifactId>-->
+<!-- <scope>provided</scope>-->
+<!-- </dependency>-->
<dependency>
<groupId>org.apache.kylin</groupId>
<artifactId>kylin-shaded-guava</artifactId>