Repository: kylin
Updated Branches:
  refs/heads/2.0-rc 2d6584ed1 -> d3b30e65d


minor, fix spark serialization issues


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/d3b30e65
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/d3b30e65
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/d3b30e65

Branch: refs/heads/2.0-rc
Commit: d3b30e65da49aea9d84387f0a0f3bf3f1045fbda
Parents: 2d6584e
Author: lidongsjtu <lid...@apache.org>
Authored: Wed Jan 20 13:12:04 2016 +0800
Committer: lidongsjtu <lid...@apache.org>
Committed: Wed Jan 20 13:12:04 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/hll/HyperLogLogPlusCounter.java    | 2 +-
 .../main/java/org/apache/kylin/engine/spark/SparkCubing.java   | 6 ++----
 2 files changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d3b30e65/core-common/src/main/java/org/apache/kylin/common/hll/HyperLogLogPlusCounter.java
----------------------------------------------------------------------
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/hll/HyperLogLogPlusCounter.java
 
b/core-common/src/main/java/org/apache/kylin/common/hll/HyperLogLogPlusCounter.java
index 11ae78f..ef91509 100644
--- 
a/core-common/src/main/java/org/apache/kylin/common/hll/HyperLogLogPlusCounter.java
+++ 
b/core-common/src/main/java/org/apache/kylin/common/hll/HyperLogLogPlusCounter.java
@@ -42,7 +42,7 @@ public class HyperLogLogPlusCounter implements Serializable, 
Comparable<HyperLog
 
     private final int p;
     private final int m;
-    private transient final HashFunction hashFunc;
+    private final HashFunction hashFunc;
     byte[] registers;
 
     public HyperLogLogPlusCounter() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/d3b30e65/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
----------------------------------------------------------------------
diff --git 
a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java 
b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
index 947ca66..488945f 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
@@ -276,7 +276,7 @@ public class SparkCubing extends AbstractApplication {
                         List<String>,
                         HashMap<Long, HyperLogLogPlusCounter>>() {
 
-                    final HashFunction hashFunction = Hashing.murmur3_32();
+                    final HashFunction hashFunction = Hashing.murmur3_128();
 
                     @Override
                     public HashMap<Long, HyperLogLogPlusCounter> 
call(HashMap<Long, HyperLogLogPlusCounter> v1, List<String> v2) throws 
Exception {
@@ -365,12 +365,11 @@ public class SparkCubing extends AbstractApplication {
                 prepare();
 
                 final CubeInstance cubeInstance = 
CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
-                final CubeDesc cubeDesc = cubeInstance.getDescriptor();
 
                 LinkedBlockingQueue<List<String>> blockingQueue = new 
LinkedBlockingQueue();
                 System.out.println("load properties finished");
                 AbstractInMemCubeBuilder inMemCubeBuilder = new 
DoggedCubeBuilder(cubeInstance.getDescriptor(), dictionaryMap);
-                final SparkCuboidWriter sparkCuboidWriter = new 
BufferedCuboidWriter(new DefaultTupleConverter(cubeSegment, columnLengthMap));
+                final SparkCuboidWriter sparkCuboidWriter = new 
BufferedCuboidWriter(new 
DefaultTupleConverter(cubeInstance.getSegmentById(segmentId), columnLengthMap));
                 
Executors.newCachedThreadPool().submit(inMemCubeBuilder.buildAsRunnable(blockingQueue,
 sparkCuboidWriter));
                 try {
                     while (listIterator.hasNext()) {
@@ -540,7 +539,6 @@ public class SparkCubing extends AbstractApplication {
         kyroClasses.addAll(new 
Reflections("org.apache.kylin.metadata.model").getSubTypesOf(Object.class));
         kyroClasses.addAll(new 
Reflections("org.apache.kylin.metadata.measure").getSubTypesOf(Object.class));
         kyroClasses.add(HashMap.class);
-        kyroClasses.add(org.apache.kylin.cube.CubeSegment.class);
         kyroClasses.add(org.apache.spark.sql.Row[].class);
         kyroClasses.add(org.apache.spark.sql.Row.class);
         
kyroClasses.add(org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema.class);

Reply via email to