You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by sh...@apache.org on 2018/08/25 00:00:37 UTC

[kylin] 02/02: KYLIN-3491 enable shrunken gd in ci test

This is an automated email from the ASF dual-hosted git repository.

shaofengshi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 80c60ee3142a1de56fc70b1d1a2f1cf57442fd21
Author: shaofengshi <sh...@apache.org>
AuthorDate: Fri Aug 24 08:38:03 2018 +0800

    KYLIN-3491 enable shrunken gd in ci test
---
 .../steps/ExtractDictionaryFromGlobalMapper.java   | 35 ++++++++++++++--------
 .../template/cube/kylin_sales_cube.json            |  8 ++---
 .../template/cube/kylin_streaming_cube.json        |  2 +-
 .../localmeta/cube_desc/ci_left_join_cube.json     |  3 +-
 4 files changed, 29 insertions(+), 19 deletions(-)

diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/ExtractDictionaryFromGlobalMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/ExtractDictionaryFromGlobalMapper.java
index 34a5ec7..be2549e 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/ExtractDictionaryFromGlobalMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/ExtractDictionaryFromGlobalMapper.java
@@ -18,13 +18,8 @@
 
 package org.apache.kylin.engine.mr.steps;
 
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -48,8 +43,13 @@ import org.apache.kylin.engine.mr.common.BatchConstants;
 import org.apache.kylin.engine.mr.common.DictionaryGetterUtil;
 import org.apache.kylin.metadata.model.TblColRef;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 public class ExtractDictionaryFromGlobalMapper<KEYIN, Object> extends KylinMapper<KEYIN, Object, Text, Text> {
     private String cubeName;
@@ -66,12 +66,13 @@ public class ExtractDictionaryFromGlobalMapper<KEYIN, Object> extends KylinMappe
     private List<Dictionary<String>> globalDicts;
 
     private String splitKey;
+    private KylinConfig config;
 
     @Override
     protected void doSetup(Context context) throws IOException {
         Configuration conf = context.getConfiguration();
         bindCurrentConfiguration(conf);
-        KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
+        config = AbstractHadoopJob.loadKylinPropsAndMetadata();
 
         cubeName = conf.get(BatchConstants.CFG_CUBE_NAME);
         cube = CubeManager.getInstance(config).getCube(cubeName);
@@ -84,14 +85,12 @@ public class ExtractDictionaryFromGlobalMapper<KEYIN, Object> extends KylinMappe
         globalColumns = cubeDesc.getAllGlobalDictColumns();
         globalColumnIndex = new int[globalColumns.size()];
         globalColumnValues = Lists.newArrayListWithExpectedSize(globalColumns.size());
-        globalDicts = Lists.newArrayListWithExpectedSize(globalColumns.size());
+
         for (int i = 0; i < globalColumns.size(); i++) {
             TblColRef colRef = globalColumns.get(i);
             int columnIndexOnFlatTbl = intermediateTableDesc.getColumnIndex(colRef);
             globalColumnIndex[i] = columnIndexOnFlatTbl;
-
             globalColumnValues.add(Sets.<String> newHashSet());
-            globalDicts.add(cubeSeg.getDictionary(colRef));
         }
 
         splitKey = DictionaryGetterUtil.getInputSplitSignature(cubeSeg, context.getInputSplit());
@@ -117,9 +116,19 @@ public class ExtractDictionaryFromGlobalMapper<KEYIN, Object> extends KylinMappe
         FileSystem fs = FileSystem.get(context.getConfiguration());
         Path outputDirBase = new Path(context.getConfiguration().get(FileOutputFormat.OUTDIR));
 
+        globalDicts = Lists.newArrayListWithExpectedSize(globalColumns.size());
+        Map<TblColRef, Dictionary<String>> dictionaryMap = cubeSeg.buildDictionaryMap();
+        for (int i = 0; i < globalColumns.size(); i++) {
+            TblColRef colRef = globalColumns.get(i);
+            globalDicts.add(dictionaryMap.get(colRef));
+        }
+
         ShrunkenDictionary.StringValueSerializer strValueSerializer = new ShrunkenDictionary.StringValueSerializer();
         for (int i = 0; i < globalColumns.size(); i++) {
             List<String> colDistinctValues = Lists.newArrayList(globalColumnValues.get(i));
+            if (colDistinctValues.size() == 0) {
+                continue;
+            }
             // sort values to accelerate the encoding process by reducing the swapping of global dictionary slices
             Collections.sort(colDistinctValues);
 
diff --git a/examples/sample_cube/template/cube/kylin_sales_cube.json b/examples/sample_cube/template/cube/kylin_sales_cube.json
index be07ffb..361a5f9 100644
--- a/examples/sample_cube/template/cube/kylin_sales_cube.json
+++ b/examples/sample_cube/template/cube/kylin_sales_cube.json
@@ -3,10 +3,10 @@
  
   "last_modified" : 0,
   "name" : "kylin_sales_cube",
-  "owner" : null,
-  "descriptor" : "kylin_sales_cube",
-  "display_name" : "kylin_sales_cube",
+  "owner" : "ADMIN",
+  "descriptor" : "kylin_sales_cube",
+  "display_name" : "kylin_sales_cube",
   "status" : "DISABLED",
   "segments" : [ ],
-  "create_time_utc" : 0
+  "create_time_utc" : 0
 }
\ No newline at end of file
diff --git a/examples/sample_cube/template/cube/kylin_streaming_cube.json b/examples/sample_cube/template/cube/kylin_streaming_cube.json
index e5286c3..ea7460d 100644
--- a/examples/sample_cube/template/cube/kylin_streaming_cube.json
+++ b/examples/sample_cube/template/cube/kylin_streaming_cube.json
@@ -2,7 +2,7 @@
   "uuid": "40a27d9d-c5f3-45c4-9b8b-513552219193",
  
   "name": "kylin_streaming_cube",
-  "owner": null,
+  "owner": "ADMIN",
   "status": "DISABLED",
   "segments": [],
   "last_modified": 0,
diff --git a/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json b/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json
index e42c522..363c680 100644
--- a/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json
+++ b/examples/test_case_data/localmeta/cube_desc/ci_left_join_cube.json
@@ -580,7 +580,8 @@
   "engine_type": 2,
   "storage_type": 2,
   "override_kylin_properties": {
-    "kylin.cube.algorithm": "INMEM"
+    "kylin.cube.algorithm": "INMEM",
+    "kylin.dictionary.shrunken-from-global-enabled": "true"
   },
   "snapshot_table_desc_list": [
     {