You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ni...@apache.org on 2020/01/03 06:32:48 UTC

[kylin] branch master updated: KYLIN-4319 in "Extract Dictionary from Global Dictionary" step, only need load global dictonaries instead of all dictonaries.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 1aa20f9  KYLIN-4319 in "Extract Dictionary from Global Dictionary" step, only need load global dictonaries instead of all dictonaries.
1aa20f9 is described below

commit 1aa20f9397fadaece33a29b4fb17a4bf8cf1e271
Author: 杨震 <ya...@migu.cn>
AuthorDate: Mon Dec 30 20:39:40 2019 +0800

    KYLIN-4319 in "Extract Dictionary from Global Dictionary" step, only need load global dictonaries instead of all dictonaries.
---
 core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java |  8 ++++++++
 .../engine/mr/steps/ExtractDictionaryFromGlobalMapper.java     | 10 ++--------
 2 files changed, 10 insertions(+), 8 deletions(-)

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 7d336fb..178fd39 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
@@ -348,6 +348,14 @@ public class CubeSegment implements IBuildable, ISegment, Serializable {
         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());
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 be2549e..7d9d51c 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
@@ -63,7 +63,6 @@ public class ExtractDictionaryFromGlobalMapper<KEYIN, Object> extends KylinMappe
     private List<TblColRef> globalColumns;
     private int[] globalColumnIndex;
     private List<Set<String>> globalColumnValues;
-    private List<Dictionary<String>> globalDicts;
 
     private String splitKey;
     private KylinConfig config;
@@ -116,12 +115,7 @@ 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));
-        }
+        Map<TblColRef, Dictionary<String>> globalDictionaryMap = cubeSeg.buildGlobalDictionaryMap(globalColumns.size());
 
         ShrunkenDictionary.StringValueSerializer strValueSerializer = new ShrunkenDictionary.StringValueSerializer();
         for (int i = 0; i < globalColumns.size(); i++) {
@@ -132,7 +126,7 @@ public class ExtractDictionaryFromGlobalMapper<KEYIN, Object> extends KylinMappe
             // sort values to accelerate the encoding process by reducing the swapping of global dictionary slices
             Collections.sort(colDistinctValues);
 
-            ShrunkenDictionaryBuilder<String> dictBuilder = new ShrunkenDictionaryBuilder<>(globalDicts.get(i));
+            ShrunkenDictionaryBuilder<String> dictBuilder = new ShrunkenDictionaryBuilder<>(globalDictionaryMap.get(globalColumns.get(i)));
             for (String colValue : colDistinctValues) {
                 dictBuilder.addValue(colValue);
             }