Repository: kylin
Updated Branches:
  refs/heads/yang21-cdh5.7 30c515265 -> 33543a90b (forced update)


KYLIN-2175 cubestatsreader support reading unfinished segments


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

Branch: refs/heads/yang21-cdh5.7
Commit: 6ba39dfe51cc2e199180d4487eb35c1c4f130719
Parents: a6e2494
Author: Hongbin Ma <mahong...@apache.org>
Authored: Fri Nov 11 11:50:25 2016 +0800
Committer: Hongbin Ma <mahong...@apache.org>
Committed: Fri Nov 11 11:50:25 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/engine/mr/common/CubeStatsReader.java  | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/6ba39dfe/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
----------------------------------------------------------------------
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
index 2dcb268..c917cfb 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
@@ -55,7 +55,6 @@ import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
 import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -149,7 +148,7 @@ public class CubeStatsReader {
     public int getMapperNumberOfFirstBuild() {
         return mapperNumberOfFirstBuild;
     }
-    
+
     public double getMapperOverlapRatioOfFirstBuild() {
         return mapperOverlapRatioOfFirstBuild;
     }
@@ -243,6 +242,7 @@ public class CubeStatsReader {
         out.println("Total estimated size(MB): " + 
SumHelper.sumDouble(cuboidSizes.values()));
         out.println("Sampling percentage:  " + samplingPercentage);
         out.println("Mapper overlap ratio: " + mapperOverlapRatioOfFirstBuild);
+        out.println("Mapper number: " + mapperNumberOfFirstBuild);
         printKVInfo(out);
         printCuboidInfoTreeEntry(cuboidRows, cuboidSizes, out);
         
out.println("----------------------------------------------------------------------------");
@@ -302,11 +302,15 @@ public class CubeStatsReader {
         System.out.println("CubeStatsReader is used to read cube statistic 
saved in metadata store");
         KylinConfig config = KylinConfig.getInstanceFromEnv();
         CubeInstance cube = CubeManager.getInstance(config).getCube(args[0]);
-        List<CubeSegment> segments = cube.getSegments(SegmentStatusEnum.READY);
+        List<CubeSegment> segments = cube.getSegments();
 
         PrintWriter out = new PrintWriter(System.out);
         for (CubeSegment seg : segments) {
-            new CubeStatsReader(seg, config).print(out);
+            try {
+                new CubeStatsReader(seg, config).print(out);
+            } catch (Exception e) {
+                logger.info("CubeStatsReader for Segment {} failed, skip it.", 
seg.getName());
+            }
         }
         out.flush();
     }

Reply via email to