You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2016/09/27 00:25:54 UTC

[01/50] [abbrv] kylin git commit: Revert "refactor BuildCubeWithStream" [Forced Update!]

Repository: kylin
Updated Branches:
  refs/heads/1.5.x-HBase1.x 6d69da921 -> 787b0aad7 (forced update)


Revert "refactor BuildCubeWithStream"

This reverts commit a08dd2e03900b321617647d1dbf1c4ee8b4b18c2.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 8e9c4550bb562b497442b17eec6485ae96e848d8
Parents: a6cd409
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:49:18 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/job/DeployUtil.java   |   7 +-
 .../kylin/provision/BuildCubeWithStream.java    |  10 +-
 .../kylin/provision/BuildCubeWithStream2.java   | 145 ++++++++++++++++++-
 3 files changed, 150 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8e9c4550/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 9e9df05..9b282e3 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -187,7 +187,6 @@ public class DeployUtil {
         File tmpFile = File.createTempFile(factTableName, "csv");
         FileOutputStream out = new FileOutputStream(tmpFile);
 
-        InputStream tempIn = null;
         try {
             if (store.exists(factTablePath)) {
                 InputStream oldContent = store.getResource(factTablePath).inputStream;
@@ -195,15 +194,13 @@ public class DeployUtil {
             }
             IOUtils.copy(in, out);
             IOUtils.closeQuietly(in);
-            IOUtils.closeQuietly(out);
 
             store.deleteResource(factTablePath);
-            tempIn = new FileInputStream(tmpFile);
-            store.putResource(factTablePath, tempIn, System.currentTimeMillis());
+            in = new FileInputStream(tmpFile);
+            store.putResource(factTablePath, in, System.currentTimeMillis());
         } finally {
             IOUtils.closeQuietly(out);
             IOUtils.closeQuietly(in);
-            IOUtils.closeQuietly(tempIn);
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8e9c4550/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index bfe1d0a..6e5313f 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -62,10 +62,10 @@ public class BuildCubeWithStream {
 
     private static final Logger logger = LoggerFactory.getLogger(org.apache.kylin.provision.BuildCubeWithStream.class);
 
-    protected CubeManager cubeManager;
+    private CubeManager cubeManager;
     private DefaultScheduler scheduler;
     protected ExecutableManager jobService;
-    static final String cubeName = "test_streaming_table_cube";
+    private static final String cubeName = "test_streaming_table_cube";
 
     private KafkaConfig kafkaConfig;
     private MockKafka kafkaServer;
@@ -114,13 +114,13 @@ public class BuildCubeWithStream {
         Assert.assertEquals(topicName, topicMetadata.topic());
     }
 
-    protected void generateStreamData(long startTime, long endTime, int numberOfRecords) throws IOException {
+    private void generateStreamData(long startTime, long endTime, int numberOfRecords) throws IOException {
         Kafka10DataLoader dataLoader = new Kafka10DataLoader(kafkaConfig);
         DeployUtil.prepareTestDataForStreamingCube(startTime, endTime, numberOfRecords, cubeName, dataLoader);
         logger.info("Test data inserted into Kafka");
     }
 
-    protected void clearSegment(String cubeName) throws Exception {
+    private void clearSegment(String cubeName) throws Exception {
         CubeInstance cube = cubeManager.getCube(cubeName);
         // remove all existing segments
         CubeUpdate cubeBuilder = new CubeUpdate(cube);
@@ -187,7 +187,7 @@ public class BuildCubeWithStream {
         return job.getId();
     }
 
-    protected ExecutableState buildSegment(String cubeName, long startOffset, long endOffset) throws Exception {
+    private ExecutableState buildSegment(String cubeName, long startOffset, long endOffset) throws Exception {
         CubeSegment segment = cubeManager.appendSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset);
         DefaultChainedExecutable job = EngineFactory.createBatchCubingJob(segment, "TEST");
         jobService.addJob(job);

http://git-wip-us.apache.org/repos/asf/kylin/blob/8e9c4550/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
index 7959701..2812446 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
@@ -18,11 +18,13 @@
 
 package org.apache.kylin.provision;
 
+import java.io.File;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.List;
 import java.util.Random;
 import java.util.TimeZone;
+import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -30,9 +32,32 @@ import java.util.concurrent.FutureTask;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Lists;
+import org.I0Itec.zkclient.ZkConnection;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kafka.common.requests.MetadataResponse;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.common.util.HBaseMetadataTestCase;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.CubeUpdate;
+import org.apache.kylin.engine.EngineFactory;
+import org.apache.kylin.engine.streaming.StreamingConfig;
+import org.apache.kylin.engine.streaming.StreamingManager;
+import org.apache.kylin.job.DeployUtil;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
+import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
+import org.apache.kylin.job.manager.ExecutableManager;
+import org.apache.kylin.job.streaming.Kafka10DataLoader;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.apache.kylin.source.kafka.KafkaConfigManager;
+import org.apache.kylin.source.kafka.config.BrokerConfig;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
+import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
 import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,12 +67,79 @@ import static java.lang.Thread.sleep;
 /**
  *  for streaming cubing case "test_streaming_table", using multiple threads to build it concurrently.
  */
-public class BuildCubeWithStream2 extends BuildCubeWithStream {
+public class BuildCubeWithStream2 {
 
     private static final Logger logger = LoggerFactory.getLogger(BuildCubeWithStream2.class);
+
+    private CubeManager cubeManager;
+    private DefaultScheduler scheduler;
+    protected ExecutableManager jobService;
+    private static final String cubeName = "test_streaming_table_cube";
+
+    private KafkaConfig kafkaConfig;
+    private MockKafka kafkaServer;
     private static boolean generateData = true;
 
-    @Override
+    public void before() throws Exception {
+        deployEnv();
+
+        final KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        jobService = ExecutableManager.getInstance(kylinConfig);
+        scheduler = DefaultScheduler.createInstance();
+        scheduler.init(new JobEngineConfig(kylinConfig), new ZookeeperJobLock());
+        if (!scheduler.hasStarted()) {
+            throw new RuntimeException("scheduler has not been started");
+        }
+        cubeManager = CubeManager.getInstance(kylinConfig);
+
+        final CubeInstance cubeInstance = CubeManager.getInstance(kylinConfig).getCube(cubeName);
+        final String factTable = cubeInstance.getFactTable();
+
+        final StreamingManager streamingManager = StreamingManager.getInstance(kylinConfig);
+        final StreamingConfig streamingConfig = streamingManager.getStreamingConfig(factTable);
+        kafkaConfig = KafkaConfigManager.getInstance(kylinConfig).getKafkaConfig(streamingConfig.getName());
+
+        String topicName = UUID.randomUUID().toString();
+        String localIp = NetworkUtils.getLocalIp();
+        BrokerConfig brokerConfig = kafkaConfig.getKafkaClusterConfigs().get(0).getBrokerConfigs().get(0);
+        brokerConfig.setHost(localIp);
+        kafkaConfig.setTopic(topicName);
+        KafkaConfigManager.getInstance(kylinConfig).saveKafkaConfig(kafkaConfig);
+
+        startEmbeddedKafka(topicName, brokerConfig);
+    }
+
+    private void startEmbeddedKafka(String topicName, BrokerConfig brokerConfig) {
+        //Start mock Kakfa
+        String zkConnectionStr = "sandbox:2181";
+        ZkConnection zkConnection = new ZkConnection(zkConnectionStr);
+        // Assert.assertEquals(ZooKeeper.States.CONNECTED, zkConnection.getZookeeperState());
+        kafkaServer = new MockKafka(zkConnection, brokerConfig.getPort(), brokerConfig.getId());
+        kafkaServer.start();
+
+        kafkaServer.createTopic(topicName, 3, 1);
+        kafkaServer.waitTopicUntilReady(topicName);
+
+        MetadataResponse.TopicMetadata topicMetadata = kafkaServer.fetchTopicMeta(topicName);
+        Assert.assertEquals(topicName, topicMetadata.topic());
+    }
+
+    private void generateStreamData(long startTime, long endTime, int numberOfRecords) throws IOException {
+        if (numberOfRecords <= 0)
+            return;
+        Kafka10DataLoader dataLoader = new Kafka10DataLoader(kafkaConfig);
+        DeployUtil.prepareTestDataForStreamingCube(startTime, endTime, numberOfRecords, cubeName, dataLoader);
+        logger.info("Test data inserted into Kafka");
+    }
+
+    private void clearSegment(String cubeName) throws Exception {
+        CubeInstance cube = cubeManager.getCube(cubeName);
+        // remove all existing segments
+        CubeUpdate cubeBuilder = new CubeUpdate(cube);
+        cubeBuilder.setToRemoveSegs(cube.getSegments().toArray(new CubeSegment[cube.getSegments().size()]));
+        cubeManager.updateCube(cubeBuilder);
+    }
+
     public void build() throws Exception {
         clearSegment(cubeName);
         SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
@@ -112,6 +204,55 @@ public class BuildCubeWithStream2 extends BuildCubeWithStream {
 
     }
 
+
+    private ExecutableState buildSegment(String cubeName, long startOffset, long endOffset) throws Exception {
+        CubeSegment segment = cubeManager.appendSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset);
+        DefaultChainedExecutable job = EngineFactory.createBatchCubingJob(segment, "TEST");
+        jobService.addJob(job);
+        waitForJob(job.getId());
+        return job.getStatus();
+    }
+
+    protected void deployEnv() throws IOException {
+        DeployUtil.overrideJobJarLocations();
+        DeployUtil.initCliWorkDir();
+        DeployUtil.deployMetadata();
+    }
+
+    public static void beforeClass() throws Exception {
+        logger.info("Adding to classpath: " + new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
+        ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
+        System.setProperty(KylinConfig.KYLIN_CONF, HBaseMetadataTestCase.SANDBOX_TEST_DATA);
+        if (StringUtils.isEmpty(System.getProperty("hdp.version"))) {
+            throw new RuntimeException("No hdp.version set; Please set hdp.version in your jvm option, for example: -Dhdp.version=2.2.4.2-2");
+        }
+        HBaseMetadataTestCase.staticCreateTestMetadata(HBaseMetadataTestCase.SANDBOX_TEST_DATA);
+    }
+
+    public static void afterClass() throws Exception {
+        HBaseMetadataTestCase.staticCleanupTestMetadata();
+    }
+
+    public void after() {
+        kafkaServer.stop();
+        DefaultScheduler.destroyInstance();
+    }
+
+    protected void waitForJob(String jobId) {
+        while (true) {
+            AbstractExecutable job = jobService.getJob(jobId);
+            if (job.getStatus() == ExecutableState.SUCCEED || job.getStatus() == ExecutableState.ERROR || job.getStatus() == ExecutableState.DISCARDED) {
+                break;
+            } else {
+                try {
+                    sleep(5000);
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
     public static void main(String[] args) throws Exception {
         try {
             beforeClass();


[31/50] [abbrv] kylin git commit: minor, dont cache rest nodes in Broadcaster

Posted by li...@apache.org.
minor, dont cache rest nodes in Broadcaster


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 5aeedd5c7d11a282cc253e13247218bb2a0e3bf3
Parents: f2732ec
Author: Yang Li <li...@apache.org>
Authored: Sat Sep 24 10:38:19 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Sep 24 10:38:38 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/metadata/cachesync/Broadcaster.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/5aeedd5c/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
index 73dd0a7..380dac8 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
@@ -77,7 +77,8 @@ public class Broadcaster {
         }
     }
 
-    public static void clearCache() {
+    // call Broadcaster.getInstance().notifyClearAll() to clear cache
+    static void clearCache() {
         CACHE.clear();
     }
 
@@ -104,7 +105,7 @@ public class Broadcaster {
             @Override
             public void run() {
                 final List<RestClient> restClients = Lists.newArrayList();
-                for (String node : nodes) {
+                for (String node : config.getRestServers()) {
                     restClients.add(new RestClient(node));
                 }
                 final ExecutorService wipingCachePool = Executors.newFixedThreadPool(restClients.size());


[14/50] [abbrv] kylin git commit: KYLIN-2029 distinct count query on lookup table

Posted by li...@apache.org.
KYLIN-2029 distinct count query on lookup table


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 472e6d547f2f04b8d6f92b9affa173a9685d3d2d
Parents: 7d716df
Author: Hongbin Ma <ma...@apache.org>
Authored: Tue Sep 20 18:19:19 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 21:17:12 2016 +0800

----------------------------------------------------------------------
 .../test/resources/query/sql_lookup/query05.sql  | 19 +++++++++++++++++++
 .../kylin/query/relnode/OLAPAggregateRel.java    |  8 ++++++--
 2 files changed, 25 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/472e6d54/kylin-it/src/test/resources/query/sql_lookup/query05.sql
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/resources/query/sql_lookup/query05.sql b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
new file mode 100644
index 0000000..31fdf61
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select count(distinct META_CATEG_NAME) as CNT ,max(META_CATEG_NAME) as y from test_category_groupings

http://git-wip-us.apache.org/repos/asf/kylin/blob/472e6d54/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index f55c86f..97efb27 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -265,7 +265,7 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
         implementor.visitChild(this, getInput());
 
         // only rewrite the innermost aggregation
-        if (!this.afterAggregate && RewriteImplementor.needRewrite(this.context)) {
+        if (!this.afterAggregate) {
             // rewrite the aggCalls
             this.rewriteAggCalls = new ArrayList<AggregateCall>(aggCalls.size());
             for (int i = 0; i < this.aggCalls.size(); i++) {
@@ -354,11 +354,15 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
         }
     }
 
+    private boolean noPrecaculatedFieldsAvailable() {
+        return !this.context.hasPrecalculatedFields() || !RewriteImplementor.needRewrite(this.context);
+    }
+
     private AggregateCall rewriteAggregateCall(AggregateCall aggCall, FunctionDesc func) {
 
         //if it's not a cube, then the "needRewriteField func" should not resort to any rewrite fields, 
         // which do not exist at all
-        if (!this.context.hasPrecalculatedFields() && func.needRewriteField()) {
+        if (noPrecaculatedFieldsAvailable() && func.needRewriteField()) {
             logger.info(func + "skip rewriteAggregateCall because no pre-aggregated field available");
             return aggCall;
         }


[24/50] [abbrv] kylin git commit: refactor: rename MeasureDecoder to MeasureCodec

Posted by li...@apache.org.
refactor: rename MeasureDecoder to MeasureCodec


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: fa5c9cb7851afff028f5a2d39d12fbbb32bd40e8
Parents: 4c9b67f
Author: Hongbin Ma <ma...@apache.org>
Authored: Thu Sep 22 11:22:32 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Thu Sep 22 18:50:35 2016 +0800

----------------------------------------------------------------------
 .../kylin/gridtable/GTAggregateScanner.java     |   8 +-
 .../metadata/measure/MeasureCodecTest.java      |   4 +-
 .../kylin/measure/BufferedMeasureCodec.java     | 107 +++++++++++++++++++
 .../kylin/measure/BufferedMeasureEncoder.java   | 107 -------------------
 .../org/apache/kylin/measure/MeasureCodec.java  | 101 +++++++++++++++++
 .../apache/kylin/measure/MeasureDecoder.java    |  97 -----------------
 .../kylin/engine/mr/common/CuboidStatsUtil.java |   4 +-
 .../engine/mr/steps/BaseCuboidMapperBase.java   |   6 +-
 .../kylin/engine/mr/steps/CuboidReducer.java    |   6 +-
 .../mr/steps/FactDistinctHiveColumnsMapper.java |   4 +-
 .../engine/mr/steps/InMemCuboidReducer.java     |   6 +-
 .../kylin/engine/mr/steps/KVGTRecordWriter.java |   4 +-
 .../engine/mr/steps/MergeCuboidMapper.java      |   6 +-
 .../kylin/engine/mr/steps/CubeReducerTest.java  |   8 +-
 .../apache/kylin/engine/spark/SparkCubing.java  |   4 +-
 .../spark/cube/DefaultTupleConverter.java       |   4 +-
 .../cardinality/ColumnCardinalityMapper.java    |   4 +-
 .../cardinality/ColumnCardinalityReducer.java   |   4 +-
 .../ColumnCardinalityReducerTest.java           |   4 +-
 .../observer/ObserverAggregators.java           |   6 +-
 .../coprocessor/endpoint/CubeVisitService.java  |   6 +-
 .../storage/hbase/steps/CubeHFileMapper.java    |   6 +-
 .../storage/hbase/steps/KeyValueCreator.java    |   6 +-
 .../storage/hbase/steps/RowValueDecoder.java    |   6 +-
 .../hbase/steps/CubeHFileMapper2Test.java       |   6 +-
 .../hbase/steps/RowValueDecoderTest.java        |   6 +-
 26 files changed, 267 insertions(+), 263 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
index db38484..9bbcd10 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -40,7 +40,7 @@ import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.MemoryBudgetController;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.MemoryBudgetController.MemoryWaterLevel;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.measure.MeasureAggregators;
 import org.apache.kylin.metadata.datatype.DataType;
@@ -178,7 +178,7 @@ public class GTAggregateScanner implements IGTScanner {
         final List<Dump> dumps;
         final int keyLength;
         final boolean[] compareMask;
-        final BufferedMeasureEncoder measureCodec;
+        final BufferedMeasureCodec measureCodec;
 
         final Comparator<byte[]> bytesComparator = new Comparator<byte[]>() {
             @Override
@@ -212,13 +212,13 @@ public class GTAggregateScanner implements IGTScanner {
             measureCodec = createMeasureCodec();
         }
 
-        private BufferedMeasureEncoder createMeasureCodec() {
+        private BufferedMeasureCodec createMeasureCodec() {
             DataType[] types = new DataType[metrics.trueBitCount()];
             for (int i = 0; i < types.length; i++) {
                 types[i] = info.getColumnType(metrics.trueBitAt(i));
             }
 
-            BufferedMeasureEncoder result = new BufferedMeasureEncoder(types);
+            BufferedMeasureCodec result = new BufferedMeasureCodec(types);
             result.setBufferSize(info.getMaxColumnLength(metrics));
             return result;
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
index 1a7e4b3..64c8970 100644
--- a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
@@ -24,7 +24,7 @@ import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.bitmap.BitmapCounter;
 import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
 import org.apache.kylin.metadata.datatype.DoubleMutable;
@@ -52,7 +52,7 @@ public class MeasureCodecTest extends LocalFileMetadataTestCase {
     @Test
     public void basicTest() {
         MeasureDesc descs[] = new MeasureDesc[] { measure("double"), measure("long"), measure("decimal"), measure("HLLC16"), measure("bitmap") };
-        BufferedMeasureEncoder codec = new BufferedMeasureEncoder(descs);
+        BufferedMeasureCodec codec = new BufferedMeasureCodec(descs);
 
         DoubleMutable d = new DoubleMutable(1.0);
         LongMutable l = new LongMutable(2);

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java b/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java
new file mode 100644
index 0000000..8a5481c
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java
@@ -0,0 +1,107 @@
+/*
+ * 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.measure;
+
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+/**
+ * This class embeds a reusable byte buffer for measure encoding, and is not thread-safe.
+ * The buffer will grow to accommodate BufferOverflowException until a limit.
+ * The problem here to solve is some measure type cannot provide accurate DataTypeSerializer.maxLength()
+ */
+@SuppressWarnings({ "unchecked" })
+public class BufferedMeasureCodec {
+    public static final int DEFAULT_BUFFER_SIZE = 1024 * 1024; // 1 MB
+    public static final int MAX_BUFFER_SIZE = 1 * 1024 * DEFAULT_BUFFER_SIZE; // 1 GB
+
+    final private MeasureCodec codec;
+
+    private ByteBuffer buf;
+    final private int[] measureSizes;
+
+    public BufferedMeasureCodec(Collection<MeasureDesc> measureDescs) {
+        this.codec = new MeasureCodec(measureDescs);
+        this.measureSizes = new int[codec.getMeasuresCount()];
+    }
+
+    public BufferedMeasureCodec(MeasureDesc... measureDescs) {
+        this.codec = new MeasureCodec(measureDescs);
+        this.measureSizes = new int[codec.getMeasuresCount()];
+    }
+
+    public BufferedMeasureCodec(DataType... dataTypes) {
+        this.codec = new MeasureCodec(dataTypes);
+        this.measureSizes = new int[codec.getMeasuresCount()];
+    }
+
+    public BufferedMeasureCodec(String... dataTypes) {
+        this.codec = new MeasureCodec(dataTypes);
+        this.measureSizes = new int[codec.getMeasuresCount()];
+    }
+
+    /** return the buffer that contains result of last encoding */
+    public ByteBuffer getBuffer() {
+        return buf;
+    }
+
+    /** return the measure sizes of last encoding */
+    public int[] getMeasureSizes() {
+        return measureSizes;
+    }
+
+    public void setBufferSize(int size) {
+        buf = null; // release memory for GC
+        buf = ByteBuffer.allocate(size);
+    }
+
+    public void decode(ByteBuffer buf, Object[] result) {
+        codec.decode(buf, result);
+    }
+
+    public ByteBuffer encode(Object[] values) {
+        if (buf == null) {
+            setBufferSize(DEFAULT_BUFFER_SIZE);
+        }
+
+        assert values.length == codec.getMeasuresCount();
+
+        while (true) {
+            try {
+                buf.clear();
+                for (int i = 0, pos = 0; i < codec.getMeasuresCount(); i++) {
+                    codec.encode(i, values[i], buf);
+                    measureSizes[i] = buf.position() - pos;
+                    pos = buf.position();
+                }
+                return buf;
+
+            } catch (BufferOverflowException boe) {
+                if (buf.capacity() >= MAX_BUFFER_SIZE)
+                    throw boe;
+
+                setBufferSize(buf.capacity() * 2);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureEncoder.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureEncoder.java b/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureEncoder.java
deleted file mode 100644
index 2b14715..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureEncoder.java
+++ /dev/null
@@ -1,107 +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.measure;
-
-import java.nio.BufferOverflowException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-/**
- * This class embeds a reusable byte buffer for measure encoding, and is not thread-safe.
- * The buffer will grow to accommodate BufferOverflowException until a limit.
- * The problem here to solve is some measure type cannot provide accurate DataTypeSerializer.maxLength()
- */
-@SuppressWarnings({ "unchecked" })
-public class BufferedMeasureEncoder {
-    public static final int DEFAULT_BUFFER_SIZE = 1024 * 1024; // 1 MB
-    public static final int MAX_BUFFER_SIZE = 1 * 1024 * DEFAULT_BUFFER_SIZE; // 1 GB
-
-    final private MeasureDecoder codec;
-
-    private ByteBuffer buf;
-    final private int[] measureSizes;
-
-    public BufferedMeasureEncoder(Collection<MeasureDesc> measureDescs) {
-        this.codec = new MeasureDecoder(measureDescs);
-        this.measureSizes = new int[codec.nMeasures];
-    }
-
-    public BufferedMeasureEncoder(MeasureDesc... measureDescs) {
-        this.codec = new MeasureDecoder(measureDescs);
-        this.measureSizes = new int[codec.nMeasures];
-    }
-
-    public BufferedMeasureEncoder(DataType... dataTypes) {
-        this.codec = new MeasureDecoder(dataTypes);
-        this.measureSizes = new int[codec.nMeasures];
-    }
-
-    public BufferedMeasureEncoder(String... dataTypes) {
-        this.codec = new MeasureDecoder(dataTypes);
-        this.measureSizes = new int[codec.nMeasures];
-    }
-
-    /** return the buffer that contains result of last encoding */
-    public ByteBuffer getBuffer() {
-        return buf;
-    }
-
-    /** return the measure sizes of last encoding */
-    public int[] getMeasureSizes() {
-        return measureSizes;
-    }
-
-    public void setBufferSize(int size) {
-        buf = null; // release memory for GC
-        buf = ByteBuffer.allocate(size);
-    }
-
-    public void decode(ByteBuffer buf, Object[] result) {
-        codec.decode(buf, result);
-    }
-
-    public ByteBuffer encode(Object[] values) {
-        if (buf == null) {
-            setBufferSize(DEFAULT_BUFFER_SIZE);
-        }
-
-        assert values.length == codec.nMeasures;
-
-        while (true) {
-            try {
-                buf.clear();
-                for (int i = 0, pos = 0; i < codec.nMeasures; i++) {
-                    codec.serializers[i].serialize(values[i], buf);
-                    measureSizes[i] = buf.position() - pos;
-                    pos = buf.position();
-                }
-                return buf;
-
-            } catch (BufferOverflowException boe) {
-                if (buf.capacity() >= MAX_BUFFER_SIZE)
-                    throw boe;
-
-                setBufferSize(buf.capacity() * 2);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
new file mode 100644
index 0000000..edaf806
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
@@ -0,0 +1,101 @@
+/*
+ * 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.measure;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+@SuppressWarnings({ "rawtypes" })
+public class MeasureCodec {
+
+    private int nMeasures;
+    private DataTypeSerializer[] serializers;
+
+    public MeasureCodec(Collection<MeasureDesc> measureDescs) {
+        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
+    }
+
+    public MeasureCodec(MeasureDesc... measureDescs) {
+        String[] dataTypes = new String[measureDescs.length];
+        for (int i = 0; i < dataTypes.length; i++) {
+            dataTypes[i] = measureDescs[i].getFunction().getReturnType();
+        }
+        init(dataTypes);
+    }
+
+    public MeasureCodec(DataType... dataTypes) {
+        init(dataTypes);
+    }
+
+    public MeasureCodec(String... dataTypes) {
+        init(dataTypes);
+    }
+
+    private void init(String[] dataTypes) {
+        DataType[] typeInstances = new DataType[dataTypes.length];
+        for (int i = 0; i < dataTypes.length; i++) {
+            typeInstances[i] = DataType.getType(dataTypes[i]);
+        }
+        init(typeInstances);
+    }
+
+    private void init(DataType[] dataTypes) {
+        nMeasures = dataTypes.length;
+        serializers = new DataTypeSerializer[nMeasures];
+
+        for (int i = 0; i < nMeasures; i++) {
+            serializers[i] = DataTypeSerializer.create(dataTypes[i]);
+        }
+    }
+
+    public void encode(int idx, Object o, ByteBuffer buf) {
+        serializers[idx].serialize(o, buf);
+    }
+
+    public int getMeasuresCount() {
+        return nMeasures;
+    }
+
+    public int[] getPeekLength(ByteBuffer buf) {
+        int[] length = new int[nMeasures];
+        int offset = 0;
+        for (int i = 0; i < nMeasures; i++) {
+            length[i] = serializers[i].peekLength(buf);
+            offset += length[i];
+            buf.position(offset);
+        }
+        return length;
+    }
+
+    public void decode(ByteBuffer buf, Object[] result) {
+        assert result.length == nMeasures;
+        for (int i = 0; i < nMeasures; i++) {
+            result[i] = serializers[i].deserialize(buf);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/core-metadata/src/main/java/org/apache/kylin/measure/MeasureDecoder.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureDecoder.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureDecoder.java
deleted file mode 100644
index 9656020..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureDecoder.java
+++ /dev/null
@@ -1,97 +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.measure;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.DataTypeSerializer;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
-/**
- * @author yangli9
- * 
- */
-@SuppressWarnings({ "rawtypes" })
-public class MeasureDecoder {
-
-    int nMeasures;
-    DataTypeSerializer[] serializers;
-
-    public MeasureDecoder(Collection<MeasureDesc> measureDescs) {
-        this((MeasureDesc[]) measureDescs.toArray(new MeasureDesc[measureDescs.size()]));
-    }
-
-    public MeasureDecoder(MeasureDesc... measureDescs) {
-        String[] dataTypes = new String[measureDescs.length];
-        for (int i = 0; i < dataTypes.length; i++) {
-            dataTypes[i] = measureDescs[i].getFunction().getReturnType();
-        }
-        init(dataTypes);
-    }
-
-    public MeasureDecoder(DataType... dataTypes) {
-        init(dataTypes);
-    }
-
-    public MeasureDecoder(String... dataTypes) {
-        init(dataTypes);
-    }
-
-    private void init(String[] dataTypes) {
-        DataType[] typeInstances = new DataType[dataTypes.length];
-        for (int i = 0; i < dataTypes.length; i++) {
-            typeInstances[i] = DataType.getType(dataTypes[i]);
-        }
-        init(typeInstances);
-    }
-
-    private void init(DataType[] dataTypes) {
-        nMeasures = dataTypes.length;
-        serializers = new DataTypeSerializer[nMeasures];
-
-        for (int i = 0; i < nMeasures; i++) {
-            serializers[i] = DataTypeSerializer.create(dataTypes[i]);
-        }
-    }
-
-    public DataTypeSerializer getSerializer(int idx) {
-        return serializers[idx];
-    }
-
-    public int[] getPeekLength(ByteBuffer buf) {
-        int[] length = new int[nMeasures];
-        int offset = 0;
-        for (int i = 0; i < nMeasures; i++) {
-            length[i] = serializers[i].peekLength(buf);
-            offset += length[i];
-            buf.position(offset);
-        }
-        return length;
-    }
-
-    public void decode(ByteBuffer buf, Object[] result) {
-        assert result.length == nMeasures;
-        for (int i = 0; i < nMeasures; i++) {
-            result[i] = serializers[i].deserialize(buf);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsUtil.java
index 87c3211..d5b4b0d 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CuboidStatsUtil.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
 
 public class CuboidStatsUtil {
@@ -50,7 +50,7 @@ public class CuboidStatsUtil {
         allCuboids.addAll(cuboidHLLMap.keySet());
         Collections.sort(allCuboids);
 
-        ByteBuffer valueBuf = ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);
+        ByteBuffer valueBuf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
         SequenceFile.Writer writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(seqFilePath), SequenceFile.Writer.keyClass(LongWritable.class), SequenceFile.Writer.valueClass(BytesWritable.class));
         try {
             // mapper overlap ratio at key -1

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
index 588b087..4f0d3fd 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
@@ -42,7 +42,7 @@ import org.apache.kylin.engine.EngineFactory;
 import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureIngester;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
@@ -76,7 +76,7 @@ public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VAL
     protected byte[][] keyBytesBuf;
     protected BytesSplitter bytesSplitter;
     protected AbstractRowKeyEncoder rowKeyEncoder;
-    protected BufferedMeasureEncoder measureCodec;
+    protected BufferedMeasureCodec measureCodec;
     private int errorRecordCounter;
     protected Text outputKey = new Text();
     protected Text outputValue = new Text();
@@ -108,7 +108,7 @@ public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<KEYIN, VAL
         bytesSplitter = new BytesSplitter(200, 16384);
         rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid);
 
-        measureCodec = new BufferedMeasureEncoder(cubeDesc.getMeasures());
+        measureCodec = new BufferedMeasureCodec(cubeDesc.getMeasures());
         measures = new Object[cubeDesc.getMeasures().size()];
 
         int colCount = cubeDesc.getRowkey().getRowKeyColumns().length;

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
index 6986344..d6e1d7e 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
@@ -29,7 +29,7 @@ import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureAggregators;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.slf4j.Logger;
@@ -47,7 +47,7 @@ public class CuboidReducer extends KylinReducer<Text, Text, Text, Text> {
     private CubeDesc cubeDesc;
     private List<MeasureDesc> measuresDescs;
 
-    private BufferedMeasureEncoder codec;
+    private BufferedMeasureCodec codec;
     private MeasureAggregators aggs;
 
     private int counter;
@@ -71,7 +71,7 @@ public class CuboidReducer extends KylinReducer<Text, Text, Text, Text> {
         cubeDesc = CubeManager.getInstance(config).getCube(cubeName).getDescriptor();
         measuresDescs = cubeDesc.getMeasures();
 
-        codec = new BufferedMeasureEncoder(measuresDescs);
+        codec = new BufferedMeasureCodec(measuresDescs);
         aggs = new MeasureAggregators(measuresDescs);
 
         input = new Object[measuresDescs.size()];

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java
index c525e90..5e278f8 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctHiveColumnsMapper.java
@@ -28,7 +28,7 @@ import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
 
 import com.google.common.collect.Lists;
@@ -159,7 +159,7 @@ public class FactDistinctHiveColumnsMapper<KEYIN> extends FactDistinctColumnsMap
     @Override
     protected void cleanup(Context context) throws IOException, InterruptedException {
         if (collectStatistics) {
-            ByteBuffer hllBuf = ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);
+            ByteBuffer hllBuf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
             // output each cuboid's hll to reducer, key is 0 - cuboidId
             HyperLogLogPlusCounter hll;
             for (int i = 0; i < cuboidIds.length; i++) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
index 54b7b7f..cfecf23 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidReducer.java
@@ -31,7 +31,7 @@ import org.apache.kylin.engine.mr.ByteArrayWritable;
 import org.apache.kylin.engine.mr.KylinReducer;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureAggregators;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.slf4j.Logger;
@@ -43,7 +43,7 @@ public class InMemCuboidReducer extends KylinReducer<ByteArrayWritable, ByteArra
 
     private static final Logger logger = LoggerFactory.getLogger(InMemCuboidReducer.class);
 
-    private BufferedMeasureEncoder codec;
+    private BufferedMeasureCodec codec;
     private MeasureAggregators aggs;
 
     private int counter;
@@ -63,7 +63,7 @@ public class InMemCuboidReducer extends KylinReducer<ByteArrayWritable, ByteArra
         CubeDesc cubeDesc = cube.getDescriptor();
 
         List<MeasureDesc> measuresDescs = cubeDesc.getMeasures();
-        codec = new BufferedMeasureEncoder(measuresDescs);
+        codec = new BufferedMeasureCodec(measuresDescs);
         aggs = new MeasureAggregators(measuresDescs);
         input = new Object[measuresDescs.size()];
         result = new Object[measuresDescs.size()];

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java
index 0c4fa56..aa323fd 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/KVGTRecordWriter.java
@@ -30,7 +30,7 @@ import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.mr.ByteArrayWritable;
 import org.apache.kylin.gridtable.GTRecord;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,7 +49,7 @@ public abstract class KVGTRecordWriter implements ICuboidWriter {
     private int measureCount;
     private byte[] keyBuf;
     private ImmutableBitSet measureColumns;
-    private ByteBuffer valueBuf = ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);
+    private ByteBuffer valueBuf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
     private ByteArrayWritable outputKey = new ByteArrayWritable();
     private ByteArrayWritable outputValue = new ByteArrayWritable();
     private long cuboidRowCount = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
index a6bef83..55b8474 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -47,7 +47,7 @@ import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureIngester;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.metadata.model.MeasureDesc;
@@ -85,7 +85,7 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
     private Map<TblColRef, Dictionary<String>> oldDicts;
     private Map<TblColRef, Dictionary<String>> newDicts;
     private List<MeasureDesc> measureDescs;
-    private BufferedMeasureEncoder codec;
+    private BufferedMeasureCodec codec;
     private Object[] measureObjs;
     private Text outputValue;
 
@@ -115,7 +115,7 @@ public class MergeCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
         rowKeyEncoderProvider = new RowKeyEncoderProvider(mergedCubeSegment);
 
         measureDescs = cubeDesc.getMeasures();
-        codec = new BufferedMeasureEncoder(measureDescs);
+        codec = new BufferedMeasureCodec(measureDescs);
         measureObjs = new Object[measureDescs.size()];
         outputValue = new Text();
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
index 3ca09cf..5e687a5 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
@@ -36,7 +36,7 @@ import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.measure.MeasureIngester;
 import org.apache.kylin.measure.MeasureType;
@@ -78,7 +78,7 @@ public class CubeReducerTest extends LocalFileMetadataTestCase {
         reduceDriver.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, "test_kylin_cube_with_slr_ready");
 
         CubeDesc cubeDesc = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_ready").getDescriptor();
-        BufferedMeasureEncoder codec = new BufferedMeasureEncoder(cubeDesc.getMeasures());
+        BufferedMeasureCodec codec = new BufferedMeasureCodec(cubeDesc.getMeasures());
 
         Text key1 = new Text("72010ustech");
         List<Text> values1 = new ArrayList<Text>();
@@ -125,7 +125,7 @@ public class CubeReducerTest extends LocalFileMetadataTestCase {
         MeasureType origMeasureType = functionDesc.getMeasureType();
         field.set(functionDesc, new MockUpMeasureType(origMeasureType));
 
-        BufferedMeasureEncoder codec = new BufferedMeasureEncoder(cubeDesc.getMeasures());
+        BufferedMeasureCodec codec = new BufferedMeasureCodec(cubeDesc.getMeasures());
 
         Text key1 = new Text("72010ustech");
         List<Text> values1 = new ArrayList<Text>();
@@ -159,7 +159,7 @@ public class CubeReducerTest extends LocalFileMetadataTestCase {
         assertTrue(result.contains(p3));
     }
 
-    private Text newValueText(BufferedMeasureEncoder codec, String sum, String min, String max, int count, int item_count) {
+    private Text newValueText(BufferedMeasureCodec codec, String sum, String min, String max, int count, int item_count) {
         Object[] values = new Object[] { new BigDecimal(sum), new BigDecimal(min), new BigDecimal(max), new LongMutable(count), new LongMutable(item_count) };
 
         ByteBuffer buf = codec.encode(values);

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/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 5c2def2..9b69063 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
@@ -81,7 +81,7 @@ import org.apache.kylin.engine.mr.common.CubeStatsReader;
 import org.apache.kylin.engine.spark.cube.BufferedCuboidWriter;
 import org.apache.kylin.engine.spark.cube.DefaultTupleConverter;
 import org.apache.kylin.engine.spark.util.IteratorUtils;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureAggregators;
 import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
 import org.apache.kylin.metadata.model.FunctionDesc;
@@ -424,7 +424,7 @@ public class SparkCubing extends AbstractApplication {
             @Override
             public Iterable<Tuple2<byte[], byte[]>> call(final Iterator<Tuple2<byte[], byte[]>> tuple2Iterator) throws Exception {
                 return new Iterable<Tuple2<byte[], byte[]>>() {
-                    final BufferedMeasureEncoder codec = new BufferedMeasureEncoder(dataTypes);
+                    final BufferedMeasureCodec codec = new BufferedMeasureCodec(dataTypes);
                     final Object[] input = new Object[measureSize];
                     final Object[] result = new Object[measureSize];
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java
index ea08f73..4f407ff 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java
@@ -29,7 +29,7 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.kv.RowKeyEncoder;
 import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
 import org.apache.kylin.gridtable.GTRecord;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import scala.Tuple2;
@@ -54,7 +54,7 @@ public final class DefaultTupleConverter implements TupleConverter {
 
     private ByteBuffer getValueBuf() {
         if (valueBuf.get() == null) {
-            valueBuf.set(ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE));
+            valueBuf.set(ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE));
         }
         return valueBuf.get();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityMapper.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityMapper.java b/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityMapper.java
index c90c62d..f27bee3 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityMapper.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityMapper.java
@@ -34,7 +34,7 @@ import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.MRUtil;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.ColumnDesc;
@@ -97,7 +97,7 @@ public class ColumnCardinalityMapper<T> extends KylinMapper<T, Object, IntWritab
     @Override
     protected void cleanup(Context context) throws IOException, InterruptedException {
         Iterator<Integer> it = hllcMap.keySet().iterator();
-        ByteBuffer buf = ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);
+        ByteBuffer buf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
         while (it.hasNext()) {
             int key = it.next();
             HyperLogLogPlusCounter hllc = hllcMap.get(key);

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducer.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducer.java b/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducer.java
index 1a7a253..858d84c 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducer.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducer.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.kylin.engine.mr.KylinReducer;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
 
 /**
@@ -79,7 +79,7 @@ public class ColumnCardinalityReducer extends KylinReducer<IntWritable, BytesWri
         while (it.hasNext()) {
             int key = it.next();
             HyperLogLogPlusCounter hllc = hllcMap.get(key);
-            ByteBuffer buf = ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);
+            ByteBuffer buf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
             buf.clear();
             hllc.writeRegisters(buf);
             buf.flip();

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/source-hive/src/test/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducerTest.java
----------------------------------------------------------------------
diff --git a/source-hive/src/test/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducerTest.java b/source-hive/src/test/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducerTest.java
index 1e45f64..d27860a 100644
--- a/source-hive/src/test/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducerTest.java
+++ b/source-hive/src/test/java/org/apache/kylin/source/hive/cardinality/ColumnCardinalityReducerTest.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.mrunit.mapreduce.ReduceDriver;
 import org.apache.hadoop.mrunit.types.Pair;
 import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.hllc.HyperLogLogPlusCounter;
 import org.junit.Before;
 import org.junit.Test;
@@ -65,7 +65,7 @@ public class ColumnCardinalityReducerTest {
             i++;
             hllc.add(Bytes.toBytes(temp));
         }
-        ByteBuffer buf = ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);
+        ByteBuffer buf = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
         buf.clear();
         hllc.writeRegisters(buf);
         buf.flip();

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
index 9ed15c6..29a30c1 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregators.java
@@ -29,7 +29,7 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesSerializer;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.MeasureTypeFactory;
@@ -241,7 +241,7 @@ public class ObserverAggregators {
         final String[] dataTypes;
         final int nMeasures;
 
-        final BufferedMeasureEncoder measureCodec;
+        final BufferedMeasureCodec measureCodec;
         final Object[] measureValues;
 
         public HCol(byte[] bFamily, byte[] bQualifier, String[] funcNames, String[] dataTypes) {
@@ -252,7 +252,7 @@ public class ObserverAggregators {
             this.nMeasures = funcNames.length;
             assert funcNames.length == dataTypes.length;
 
-            this.measureCodec = new BufferedMeasureEncoder(dataTypes);
+            this.measureCodec = new BufferedMeasureCodec(dataTypes);
             this.measureValues = new Object[nMeasures];
         }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index 13a7b53..4790d6e 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -55,7 +55,7 @@ import org.apache.kylin.gridtable.GTScanTimeoutException;
 import org.apache.kylin.gridtable.IGTScanner;
 import org.apache.kylin.gridtable.IGTStore;
 import org.apache.kylin.gridtable.StorageSideBehavior;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -290,9 +290,9 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
             IGTScanner rawScanner = store.scan(scanReq);
             IGTScanner finalScanner = scanReq.decorateScanner(rawScanner, behavior.filterToggledOn(), behavior.aggrToggledOn(), deadline);
 
-            ByteBuffer buffer = ByteBuffer.allocate(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);
+            ByteBuffer buffer = ByteBuffer.allocate(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);
 
-            ByteArrayOutputStream outputStream = new ByteArrayOutputStream(BufferedMeasureEncoder.DEFAULT_BUFFER_SIZE);//ByteArrayOutputStream will auto grow
+            ByteArrayOutputStream outputStream = new ByteArrayOutputStream(BufferedMeasureCodec.DEFAULT_BUFFER_SIZE);//ByteArrayOutputStream will auto grow
             int finalRowCount = 0;
 
             try {

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java
index 7523249..8205ff7 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper.java
@@ -33,7 +33,7 @@ import org.apache.kylin.cube.model.HBaseColumnFamilyDesc;
 import org.apache.kylin.engine.mr.KylinMapper;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.MeasureDecoder;
+import org.apache.kylin.measure.MeasureCodec;
 
 import com.google.common.collect.Lists;
 
@@ -48,7 +48,7 @@ public class CubeHFileMapper extends KylinMapper<Text, Text, ImmutableBytesWrita
     String cubeName;
     CubeDesc cubeDesc;
 
-    MeasureDecoder inputCodec;
+    MeasureCodec inputCodec;
     Object[] inputMeasures;
     List<KeyValueCreator> keyValueCreators;
 
@@ -62,7 +62,7 @@ public class CubeHFileMapper extends KylinMapper<Text, Text, ImmutableBytesWrita
         CubeManager cubeMgr = CubeManager.getInstance(config);
         cubeDesc = cubeMgr.getCube(cubeName).getDescriptor();
 
-        inputCodec = new MeasureDecoder(cubeDesc.getMeasures());
+        inputCodec = new MeasureCodec(cubeDesc.getMeasures());
         inputMeasures = new Object[cubeDesc.getMeasures().size()];
         keyValueCreators = Lists.newArrayList();
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java
index 490031e..222c9f4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/KeyValueCreator.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.metadata.model.MeasureDesc;
 
 /**
@@ -40,7 +40,7 @@ public class KeyValueCreator {
     int[] refIndex;
     MeasureDesc[] refMeasures;
 
-    BufferedMeasureEncoder codec;
+    BufferedMeasureCodec codec;
     Object[] colValues;
 
     public boolean isFullCopy;
@@ -54,7 +54,7 @@ public class KeyValueCreator {
         refIndex = colDesc.getMeasureIndex();
         refMeasures = colDesc.getMeasures();
 
-        codec = new BufferedMeasureEncoder(refMeasures);
+        codec = new BufferedMeasureCodec(refMeasures);
         colValues = new Object[refMeasures.length];
 
         isFullCopy = true;

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java
index 2aa285c..86104e2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/RowValueDecoder.java
@@ -25,7 +25,7 @@ import java.util.Collection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.measure.MeasureDecoder;
+import org.apache.kylin.measure.MeasureCodec;
 import org.apache.kylin.metadata.datatype.DoubleMutable;
 import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.FunctionDesc;
@@ -44,7 +44,7 @@ public class RowValueDecoder implements Cloneable {
     private final byte[] hbaseColumnFamily;
     private final byte[] hbaseColumnQualifier;
 
-    private final MeasureDecoder codec;
+    private final MeasureCodec codec;
     private final BitSet projectionIndex;
     private final MeasureDesc[] measures;
     private final Object[] values;
@@ -55,7 +55,7 @@ public class RowValueDecoder implements Cloneable {
         this.hbaseColumnQualifier = Bytes.toBytes(hbaseColumn.getQualifier());
         this.projectionIndex = new BitSet();
         this.measures = hbaseColumn.getMeasures();
-        this.codec = new MeasureDecoder(measures);
+        this.codec = new MeasureCodec(measures);
         this.values = new Object[measures.length];
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.java
index c1fd2e2..77b18e2 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/CubeHFileMapper2Test.java
@@ -33,7 +33,7 @@ import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.engine.mr.HadoopUtil;
-import org.apache.kylin.measure.MeasureDecoder;
+import org.apache.kylin.measure.MeasureCodec;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -47,7 +47,7 @@ public class CubeHFileMapper2Test extends LocalFileMetadataTestCase {
 
     String cubeName = "test_kylin_cube_with_slr_ready";
 
-    MeasureDecoder codec;
+    MeasureCodec codec;
     Object[] outKV = new Object[2];
 
     @Before
@@ -57,7 +57,7 @@ public class CubeHFileMapper2Test extends LocalFileMetadataTestCase {
         FileUtils.deleteDirectory(new File("../job/meta"));
         FileUtils.copyDirectory(new File(getTestConfig().getMetadataUrl()), new File("../job/meta"));
         CubeDesc desc = CubeManager.getInstance(getTestConfig()).getCube(cubeName).getDescriptor();
-        codec = new MeasureDecoder(desc.getMeasures());
+        codec = new MeasureCodec(desc.getMeasures());
     }
 
     @After

http://git-wip-us.apache.org/repos/asf/kylin/blob/fa5c9cb7/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
index 81ac32f..f0ad6e0 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
@@ -29,7 +29,7 @@ import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
-import org.apache.kylin.measure.BufferedMeasureEncoder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.FunctionDesc;
@@ -56,7 +56,7 @@ public class RowValueDecoderTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_ready").getDescriptor();
         HBaseColumnDesc hbaseCol = cubeDesc.getHbaseMapping().getColumnFamily()[0].getColumns()[0];
 
-        BufferedMeasureEncoder codec = new BufferedMeasureEncoder(hbaseCol.getMeasures());
+        BufferedMeasureCodec codec = new BufferedMeasureCodec(hbaseCol.getMeasures());
         BigDecimal sum = new BigDecimal("333.1234567");
         BigDecimal min = new BigDecimal("333.1111111");
         BigDecimal max = new BigDecimal("333.1999999");
@@ -86,7 +86,7 @@ public class RowValueDecoderTest extends LocalFileMetadataTestCase {
         CubeDesc cubeDesc = CubeManager.getInstance(getTestConfig()).getCube("test_kylin_cube_with_slr_ready").getDescriptor();
         HBaseColumnDesc hbaseCol = cubeDesc.getHbaseMapping().getColumnFamily()[0].getColumns()[0];
 
-        BufferedMeasureEncoder codec = new BufferedMeasureEncoder(hbaseCol.getMeasures());
+        BufferedMeasureCodec codec = new BufferedMeasureCodec(hbaseCol.getMeasures());
         BigDecimal sum = new BigDecimal("11111111111111111111333.1234567");
         BigDecimal min = new BigDecimal("333.1111111");
         BigDecimal max = new BigDecimal("333.1999999");


[12/50] [abbrv] kylin git commit: minor, add test data provision tool for dev purpose

Posted by li...@apache.org.
minor, add test data provision tool for dev purpose


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: f62541ed4794adacaca508da5a4997dea82bd385
Parents: dbc02b7
Author: Hongbin Ma <ma...@apache.org>
Authored: Tue Sep 20 20:46:47 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 21:17:12 2016 +0800

----------------------------------------------------------------------
 .../kylin/provision/ProvisionWithoutBuild.java  | 29 ++++++++++++++++++++
 1 file changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/f62541ed/kylin-it/src/test/java/org/apache/kylin/provision/ProvisionWithoutBuild.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/ProvisionWithoutBuild.java b/kylin-it/src/test/java/org/apache/kylin/provision/ProvisionWithoutBuild.java
new file mode 100644
index 0000000..71bb0ca
--- /dev/null
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/ProvisionWithoutBuild.java
@@ -0,0 +1,29 @@
+/*
+ * 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.provision;
+
+import org.apache.kylin.job.DeployUtil;
+
+public class ProvisionWithoutBuild extends BuildCubeWithEngine {
+    public static void main(String[] args) throws Exception {
+        beforeClass();
+        DeployUtil.deployMetadata();
+        DeployUtil.prepareTestDataForNormalCubes("test_kylin_cube_with_slr_empty");
+    }
+}


[26/50] [abbrv] kylin git commit: KYLIN-2033 refactor metadata sync mechanismn

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java
index d594873..2de8527 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java
@@ -44,9 +44,10 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
 import org.apache.kylin.metadata.MetadataConstants;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
+import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -73,8 +74,26 @@ public class KafkaConfigManager {
 
     private KafkaConfigManager(KylinConfig config) throws IOException {
         this.config = config;
-        this.kafkaMap = new CaseInsensitiveStringCache<KafkaConfig>(config, Broadcaster.TYPE.KAFKA);
+        this.kafkaMap = new CaseInsensitiveStringCache<KafkaConfig>(config, "kafka");
+        
+        // touch lower level metadata before registering my listener
         reloadAllKafkaConfig();
+        Broadcaster.getInstance(config).registerListener(new KafkaSyncListener(), "kafka");
+    }
+
+    private class KafkaSyncListener extends Broadcaster.Listener {
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            if (event == Event.DROP)
+                removeKafkaConfigLocal(cacheKey);
+            else
+                reloadKafkaConfigLocal(cacheKey);
+        }
     }
 
     private ResourceStore getStore() {
@@ -199,6 +218,10 @@ public class KafkaConfigManager {
         kafkaMap.remove(kafkaConfig.getName());
     }
 
+    private void removeKafkaConfigLocal(String name) {
+        kafkaMap.remove(name);
+    }
+    
     private void reloadAllKafkaConfig() throws IOException {
         ResourceStore store = getStore();
         logger.info("Reloading Kafka Metadata from folder " + store.getReadableResourcePath(ResourceStore.KAFKA_RESOURCE_ROOT));

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index 3066fb5..52aa7ea 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -44,7 +44,6 @@ import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
 import org.apache.kylin.common.restclient.RestClient;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Dictionary;
@@ -57,6 +56,7 @@ import org.apache.kylin.dict.DictionaryManager;
 import org.apache.kylin.dict.lookup.SnapshotManager;
 import org.apache.kylin.dict.lookup.SnapshotTable;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TableDesc;
@@ -564,7 +564,7 @@ public class CubeMigrationCLI {
             RestClient restClient = new RestClient(node);
             try {
                 logger.info("update meta cache for " + node);
-                restClient.wipeCache(Broadcaster.TYPE.ALL.getType(), Broadcaster.EVENT.UPDATE.getType(), "all");
+                restClient.wipeCache(Broadcaster.SYNC_ALL, Broadcaster.Event.UPDATE.getType(), Broadcaster.SYNC_ALL);
             } catch (IOException e) {
                 logger.error(e.getMessage());
             }


[36/50] [abbrv] kylin git commit: KYLIN-2033 code review and bug fixes

Posted by li...@apache.org.
KYLIN-2033 code review and bug fixes


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: bf127a9167c506025616031374a91c57eb1355ed
Parents: afa350f
Author: Yang Li <li...@apache.org>
Authored: Sat Sep 24 14:29:39 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Sep 24 14:29:56 2016 +0800

----------------------------------------------------------------------
 .../kylin/metadata/cachesync/Broadcaster.java   |  4 ++--
 .../realization/RealizationRegistry.java        |  8 ++++++++
 .../kylin/rest/controller/CacheController.java  | 17 +++++++++-------
 .../apache/kylin/rest/service/AdminService.java |  1 -
 .../apache/kylin/rest/service/CacheService.java | 21 ++++++++++++++++----
 .../apache/kylin/rest/service/QueryService.java |  4 +---
 webapp/app/js/services/cache.js                 |  2 +-
 7 files changed, 39 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
index 380dac8..75b2333 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
@@ -112,7 +112,7 @@ public class Broadcaster {
                 while (true) {
                     try {
                         final BroadcastEvent broadcastEvent = broadcastEvents.takeFirst();
-                        logger.info("Announcing new broadcast event:" + broadcastEvent);
+                        logger.info("Announcing new broadcast event: " + broadcastEvent);
                         for (final RestClient restClient : restClients) {
                             wipingCachePool.execute(new Runnable() {
                                 @Override
@@ -326,7 +326,7 @@ public class Broadcaster {
 
         @Override
         public String toString() {
-            return Objects.toStringHelper(this).add("type", entity).add("name", cacheKey).add("action", event).toString();
+            return Objects.toStringHelper(this).add("entity", entity).add("event", event).add("cacheKey", cacheKey).toString();
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
index e6b1a86..77e2679 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
@@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -75,6 +76,13 @@ public class RealizationRegistry {
         logger.info("Initializing RealizationRegistry with metadata url " + config);
         this.config = config;
         init();
+
+        Broadcaster.getInstance(config).registerListener(new Broadcaster.Listener() {
+            @Override
+            public void onClearAll(Broadcaster broadcaster) throws IOException {
+                clearCache();
+            }
+        }, "");
     }
 
     private void init() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
index 667046b..254aabf 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
@@ -47,13 +47,16 @@ public class CacheController extends BasicController {
     private CacheService cacheService;
 
     /**
-     * Wipe system cache
-     *
-     * @param entity  {@link Broadcaster.TYPE}
-     * @param event {@link Broadcaster.Event}
-     * @param cacheKey
-     * @return if the action success
-     * @throws IOException
+     * Announce wipe cache to all cluster nodes
+     */
+    @RequestMapping(value = "/announce/{entity}/{cacheKey}/{event}", method = { RequestMethod.PUT })
+    @ResponseBody
+    public void announceWipeCache(@PathVariable String entity, @PathVariable String event, @PathVariable String cacheKey) throws IOException {
+        cacheService.annouceWipeCache(entity, event, cacheKey);
+    }
+
+    /**
+     * Wipe cache on this node
      */
     @RequestMapping(value = "/{entity}/{cacheKey}/{event}", method = { RequestMethod.PUT })
     @ResponseBody

http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java b/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java
index b92c305..1a94967 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/AdminService.java
@@ -48,7 +48,6 @@ public class AdminService extends BasicService {
      */
     @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN)
     public String getEnv() {
-        logger.debug("Get Kylin Runtime environment");
         PropertiesConfiguration tempConfig = new PropertiesConfiguration();
 
         // Add Java Env

http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
index 5d29dcd..c9c2dd7 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
@@ -59,15 +59,23 @@ public class CacheService extends BasicService {
 
     @Autowired
     private CacheManager cacheManager;
-    
+
     private Broadcaster.Listener cacheSyncListener = new Broadcaster.Listener() {
         @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            removeAllOLAPDataSources();
+            cleanAllDataCache();
+        }
+
+        @Override
         public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException {
             removeOLAPDataSource(project);
+            cleanDataCache(project);
         }
 
         @Override
         public void onProjectDataChange(Broadcaster broadcaster, String project) throws IOException {
+            removeOLAPDataSource(project); // data availability (cube enabled/disabled) affects exposed schema to SQL
             cleanDataCache(project);
         }
 
@@ -84,7 +92,7 @@ public class CacheService extends BasicService {
                             logger.error("Error in updateOnNewSegmentReady()", ex);
                         }
                     }
-                }.run();
+                }.start();
             }
         }
     };
@@ -94,12 +102,17 @@ public class CacheService extends BasicService {
         this.cubeService = cubeService;
     }
 
+    public void annouceWipeCache(String entity, String event, String cacheKey) {
+        Broadcaster broadcaster = Broadcaster.getInstance(getConfig());
+        broadcaster.queue(entity, event, cacheKey);
+    }
+
     public void notifyMetadataChange(String entity, Event event, String cacheKey) throws IOException {
         Broadcaster broadcaster = Broadcaster.getInstance(getConfig());
-        
+
         // broadcaster can be clearCache() too, make sure listener is registered; re-registration will be ignored
         broadcaster.registerListener(cacheSyncListener, "cube");
-        
+
         broadcaster.notifyListener(entity, event, cacheKey);
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index df296cf..8303cee 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -337,7 +337,6 @@ public class QueryService extends BasicService {
             conn = dataSource.getConnection();
             DatabaseMetaData metaData = conn.getMetaData();
 
-            logger.debug("getting table metas");
             JDBCTableMeta = metaData.getTables(null, null, null, null);
 
             tableMetas = new LinkedList<TableMeta>();
@@ -355,7 +354,6 @@ public class QueryService extends BasicService {
                 }
             }
 
-            logger.debug("getting column metas");
             columnMeta = metaData.getColumns(null, null, null, null);
 
             while (columnMeta.next()) {
@@ -369,7 +367,7 @@ public class QueryService extends BasicService {
                     tableMap.get(colmnMeta.getTABLE_SCHEM() + "#" + colmnMeta.getTABLE_NAME()).addColumn(colmnMeta);
                 }
             }
-            logger.debug("done column metas");
+            
         } finally {
             close(columnMeta, null, conn);
             if (JDBCTableMeta != null) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/webapp/app/js/services/cache.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/services/cache.js b/webapp/app/js/services/cache.js
index b61a179..bcec603 100644
--- a/webapp/app/js/services/cache.js
+++ b/webapp/app/js/services/cache.js
@@ -17,7 +17,7 @@
  */
 
 KylinApp.factory('CacheService', ['$resource', function ($resource, config) {
-  return $resource(Config.service.url + 'cache/:type/:name/:action', {}, {
+  return $resource(Config.service.url + 'cache/announce/:type/:name/:action', {}, {
     clean: {method: 'PUT', params: {type: 'all', name: 'all', action: 'update'}, isArray: false}
   });
 }]);


[06/50] [abbrv] kylin git commit: Revert "KYLIN-1726 allow job discard itself"

Posted by li...@apache.org.
Revert "KYLIN-1726 allow job discard itself"

This reverts commit aff2df5987e98ee9fd64d4803a8a2dea90013e40.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 3ae2549ba89e3a2c8ed94a2089678227cf78312d
Parents: 96d5f0e
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:49:57 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/cube/CubeManager.java | 14 +++++--
 .../kylin/job/execution/AbstractExecutable.java |  2 -
 .../job/execution/DefaultChainedExecutable.java |  2 -
 .../kylin/job/execution/ExecuteResult.java      |  4 --
 .../kylin/job/DiscardedTestExecutable.java      | 41 --------------------
 .../impl/threadpool/DefaultSchedulerTest.java   | 16 --------
 6 files changed, 11 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/3ae2549b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
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 d494fcc..11eabce 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -322,26 +322,34 @@ public class CubeManager implements IRealizationProvider {
             Iterator<CubeSegment> iterator = newSegs.iterator();
             while (iterator.hasNext()) {
                 CubeSegment currentSeg = iterator.next();
+                boolean found = false;
                 for (CubeSegment toRemoveSeg : update.getToRemoveSegs()) {
                     if (currentSeg.getUuid().equals(toRemoveSeg.getUuid())) {
-                        logger.info("Remove segment " + currentSeg.toString());
-                        toRemoveResources.add(currentSeg.getStatisticsResourcePath());
                         iterator.remove();
-                        break;
+                        toRemoveResources.add(toRemoveSeg.getStatisticsResourcePath());
+                        found = true;
                     }
                 }
+                if (found == false) {
+                    logger.error("Segment '" + currentSeg.getName() + "' doesn't exist for remove.");
+                }
             }
 
         }
 
         if (update.getToUpdateSegs() != null) {
             for (CubeSegment segment : update.getToUpdateSegs()) {
+                boolean found = false;
                 for (int i = 0; i < newSegs.size(); i++) {
                     if (newSegs.get(i).getUuid().equals(segment.getUuid())) {
                         newSegs.set(i, segment);
+                        found = true;
                         break;
                     }
                 }
+                if (found == false) {
+                    logger.error("Segment '" + segment.getName() + "' doesn't exist for update.");
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/3ae2549b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
index b4ca469..90e4d3c 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
@@ -74,8 +74,6 @@ public abstract class AbstractExecutable implements Executable, Idempotent {
         if (!isDiscarded()) {
             if (result.succeed()) {
                 executableManager.updateJobOutput(getId(), ExecutableState.SUCCEED, null, result.output());
-            } else if (result.discarded()) {
-                executableManager.updateJobOutput(getId(), ExecutableState.DISCARDED, null, result.output());
             } else {
                 executableManager.updateJobOutput(getId(), ExecutableState.ERROR, null, result.output());
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/3ae2549b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
index 5a57b05..39a5f4f 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
@@ -119,8 +119,6 @@ public class DefaultChainedExecutable extends AbstractExecutable implements Chai
             } else {
                 jobService.updateJobOutput(getId(), ExecutableState.READY, null, null);
             }
-        } else if (result.discarded()) {
-            jobService.updateJobOutput(getId(), ExecutableState.DISCARDED, null, result.output());
         } else {
             setEndTime(System.currentTimeMillis());
             jobService.updateJobOutput(getId(), ExecutableState.ERROR, null, result.output());

http://git-wip-us.apache.org/repos/asf/kylin/blob/3ae2549b/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java b/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
index 2347e7d..760a574 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/ExecuteResult.java
@@ -49,10 +49,6 @@ public final class ExecuteResult {
         return state == State.SUCCEED;
     }
 
-    public boolean discarded() {
-        return state == State.DISCARDED;
-    }
-
     public String output() {
         return output;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/3ae2549b/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java b/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java
deleted file mode 100644
index 9362e18..0000000
--- a/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java
+++ /dev/null
@@ -1,41 +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.job;
-
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-
-/**
- */
-public class DiscardedTestExecutable extends BaseTestExecutable {
-
-    public DiscardedTestExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException e) {
-        }
-        return new ExecuteResult(ExecuteResult.State.DISCARDED, "discarded");
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/3ae2549b/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/DefaultSchedulerTest.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/DefaultSchedulerTest.java b/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/DefaultSchedulerTest.java
index 2baf10a..df521f9 100644
--- a/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/DefaultSchedulerTest.java
+++ b/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/DefaultSchedulerTest.java
@@ -29,7 +29,6 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.kylin.job.DiscardedTestExecutable;
 import org.apache.kylin.job.BaseTestExecutable;
 import org.apache.kylin.job.ErrorTestExecutable;
 import org.apache.kylin.job.FailedTestExecutable;
@@ -84,21 +83,6 @@ public class DefaultSchedulerTest extends BaseSchedulerTest {
     }
 
     @Test
-    public void testSucceedAndDiscarded() throws Exception {
-        DefaultChainedExecutable job = new DefaultChainedExecutable();
-        BaseTestExecutable task1 = new SucceedTestExecutable();
-        BaseTestExecutable task2 = new DiscardedTestExecutable();
-        job.addTask(task1);
-        job.addTask(task2);
-        jobService.addJob(job);
-        waitForJobFinish(job.getId());
-        Assert.assertEquals(ExecutableState.DISCARDED, jobService.getOutput(job.getId()).getState());
-        Assert.assertEquals(ExecutableState.SUCCEED, jobService.getOutput(task1.getId()).getState());
-        Assert.assertEquals(ExecutableState.DISCARDED, jobService.getOutput(task2.getId()).getState());
-    }
-
-
-    @Test
     public void testSucceedAndError() throws Exception {
         DefaultChainedExecutable job = new DefaultChainedExecutable();
         BaseTestExecutable task1 = new ErrorTestExecutable();


[27/50] [abbrv] kylin git commit: KYLIN-2033 refactor metadata sync mechanismn

Posted by li...@apache.org.
KYLIN-2033 refactor metadata sync mechanismn


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 72005ea5b7e945268028b88e5fc8f197d7608861
Parents: 858fad6
Author: Li Yang <li...@apache.org>
Authored: Thu Sep 22 18:42:37 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Fri Sep 23 22:15:58 2016 +0800

----------------------------------------------------------------------
 .../common/restclient/AbstractRestCache.java    |  52 ---
 .../kylin/common/restclient/Broadcaster.java    | 272 ---------------
 .../restclient/CaseInsensitiveStringCache.java  |  42 ---
 .../kylin/common/restclient/RestClient.java     |   4 +-
 .../common/restclient/SingleValueCache.java     | 103 ------
 .../common/util/AbstractKylinTestCase.java      |   1 -
 .../org/apache/kylin/cube/CubeDescManager.java  |  52 ++-
 .../org/apache/kylin/cube/CubeInstance.java     |   4 -
 .../java/org/apache/kylin/cube/CubeManager.java |  41 ++-
 .../apache/kylin/metadata/MetadataManager.java  | 107 +++++-
 .../kylin/metadata/cachesync/AbstractCache.java |  50 +++
 .../kylin/metadata/cachesync/Broadcaster.java   | 332 +++++++++++++++++++
 .../cachesync/CaseInsensitiveStringCache.java   |  42 +++
 .../metadata/cachesync/SingleValueCache.java    | 103 ++++++
 .../kylin/metadata/project/ProjectManager.java  |  50 ++-
 .../kylin/storage/hybrid/HybridManager.java     |  57 +++-
 .../engine/streaming/StreamingManager.java      |  25 +-
 .../kylin/rest/controller/CacheController.java  |  33 +-
 .../apache/kylin/rest/service/CacheService.java | 214 +++---------
 .../kylin/rest/service/CacheServiceTest.java    |  32 +-
 .../kylin/rest/service/ServiceTestBase.java     |  15 +-
 .../kylin/source/kafka/KafkaConfigManager.java  |  29 +-
 .../storage/hbase/util/CubeMigrationCLI.java    |   4 +-
 23 files changed, 920 insertions(+), 744 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-common/src/main/java/org/apache/kylin/common/restclient/AbstractRestCache.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/AbstractRestCache.java b/core-common/src/main/java/org/apache/kylin/common/restclient/AbstractRestCache.java
deleted file mode 100644
index 584131d..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/AbstractRestCache.java
+++ /dev/null
@@ -1,52 +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.common.restclient;
-
-import org.apache.kylin.common.KylinConfig;
-
-/**
- * @author xjiang
- * 
- */
-public abstract class AbstractRestCache<K, V> {
-
-    protected final KylinConfig config;
-    protected final Broadcaster.TYPE syncType;
-
-    protected AbstractRestCache(KylinConfig config, Broadcaster.TYPE syncType) {
-        this.config = config;
-        this.syncType = syncType;
-    }
-
-    public Broadcaster getBroadcaster() {
-        return Broadcaster.getInstance(config);
-    }
-
-    public abstract void put(K key, V value);
-
-    public abstract void putLocal(K key, V value);
-
-    public abstract void remove(K key);
-
-    public abstract void removeLocal(K key);
-
-    public abstract void clear();
-
-    public abstract int size();
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-common/src/main/java/org/apache/kylin/common/restclient/Broadcaster.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/Broadcaster.java b/core-common/src/main/java/org/apache/kylin/common/restclient/Broadcaster.java
deleted file mode 100644
index 230888f..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/Broadcaster.java
+++ /dev/null
@@ -1,272 +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.common.restclient;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.BlockingDeque;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.DaemonThreadFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Objects;
-import com.google.common.collect.Lists;
-
-/**
- * Broadcast kylin event out
- */
-public class Broadcaster {
-
-    private static final Logger logger = LoggerFactory.getLogger(Broadcaster.class);
-
-    // static cached instances
-    private static final ConcurrentHashMap<KylinConfig, Broadcaster> CACHE = new ConcurrentHashMap<KylinConfig, Broadcaster>();
-
-    public static Broadcaster getInstance(KylinConfig config) {
-        Broadcaster r = CACHE.get(config);
-        if (r != null) {
-            return r;
-        }
-
-        synchronized (Broadcaster.class) {
-            r = CACHE.get(config);
-            if (r != null) {
-                return r;
-            }
-
-            r = new Broadcaster(config);
-            CACHE.put(config, r);
-            if (CACHE.size() > 1) {
-                logger.warn("More than one singleton exist");
-            }
-            return r;
-        }
-    }
-
-    public static void clearCache() {
-        CACHE.clear();
-    }
-
-    // ============================================================================
-
-    private BlockingDeque<BroadcastEvent> broadcastEvents = new LinkedBlockingDeque<>();
-
-    private AtomicLong counter = new AtomicLong();
-
-    private Broadcaster(final KylinConfig config) {
-        final String[] nodes = config.getRestServers();
-        if (nodes == null || nodes.length < 1) {
-            logger.warn("There is no available rest server; check the 'kylin.rest.servers' config");
-            broadcastEvents = null; // disable the broadcaster
-            return;
-        }
-        logger.debug(nodes.length + " nodes in the cluster: " + Arrays.toString(nodes));
-
-        Executors.newSingleThreadExecutor(new DaemonThreadFactory()).execute(new Runnable() {
-            @Override
-            public void run() {
-                final List<RestClient> restClients = Lists.newArrayList();
-                for (String node : nodes) {
-                    restClients.add(new RestClient(node));
-                }
-                final ExecutorService wipingCachePool = Executors.newFixedThreadPool(restClients.size());
-                while (true) {
-                    try {
-                        final BroadcastEvent broadcastEvent = broadcastEvents.takeFirst();
-                        logger.info("new broadcast event:" + broadcastEvent);
-                        for (final RestClient restClient : restClients) {
-                            wipingCachePool.execute(new Runnable() {
-                                @Override
-                                public void run() {
-                                    try {
-                                        restClient.wipeCache(broadcastEvent.getType(), broadcastEvent.getAction(), broadcastEvent.getName());
-                                    } catch (IOException e) {
-                                        logger.warn("Thread failed during wipe cache at " + broadcastEvent);
-                                    }
-                                }
-                            });
-                        }
-                    } catch (Exception e) {
-                        logger.error("error running wiping", e);
-                    }
-                }
-            }
-        });
-    }
-
-    /**
-     * Broadcast the cubedesc event out
-     * 
-     * @param action
-     *            event action
-     */
-    public void queue(String type, String action, String key) {
-        if (broadcastEvents == null)
-            return;
-
-        try {
-            counter.incrementAndGet();
-            broadcastEvents.putFirst(new BroadcastEvent(type, action, key));
-        } catch (Exception e) {
-            counter.decrementAndGet();
-            logger.error("error putting BroadcastEvent", e);
-        }
-    }
-
-    public long getCounterAndClear() {
-        return counter.getAndSet(0);
-    }
-
-    public enum EVENT {
-
-        CREATE("create"), UPDATE("update"), DROP("drop");
-        private String text;
-
-        EVENT(String text) {
-            this.text = text;
-        }
-
-        public String getType() {
-            return text;
-        }
-
-        public static EVENT getEvent(String event) {
-            for (EVENT one : values()) {
-                if (one.getType().equalsIgnoreCase(event)) {
-                    return one;
-                }
-            }
-
-            return null;
-        }
-    }
-
-    public enum TYPE {
-        ALL("all"), //
-        PROJECT("project"), //
-        CUBE("cube"), //
-        CUBE_DESC("cube_desc"), //
-        STREAMING("streaming"), //
-        KAFKA("kafka"), //
-        INVERTED_INDEX("inverted_index"), //
-        INVERTED_INDEX_DESC("ii_desc"), // 
-        TABLE("table"), //
-        DATA_MODEL("data_model"), //
-        EXTERNAL_FILTER("external_filter"), //
-        HYBRID("hybrid");
-        
-        private String text;
-
-        TYPE(String text) {
-            this.text = text;
-        }
-
-        public String getType() {
-            return text;
-        }
-
-        /**
-         * @param type
-         * @return
-         */
-        public static TYPE getType(String type) {
-            for (TYPE one : values()) {
-                if (one.getType().equalsIgnoreCase(type)) {
-                    return one;
-                }
-            }
-
-            return null;
-        }
-    }
-
-    public static class BroadcastEvent {
-        private String type;
-        private String action;
-        private String name;
-
-        public BroadcastEvent(String type, String action, String name) {
-            super();
-            this.type = type;
-            this.action = action;
-            this.name = name;
-        }
-
-        public String getType() {
-            return type;
-        }
-
-        public String getAction() {
-            return action;
-        }
-
-        public String getName() {
-            return name;
-        }
-
-        @Override
-        public int hashCode() {
-            final int prime = 31;
-            int result = 1;
-            result = prime * result + ((action == null) ? 0 : action.hashCode());
-            result = prime * result + ((name == null) ? 0 : name.hashCode());
-            result = prime * result + ((type == null) ? 0 : type.hashCode());
-            return result;
-        }
-
-        @Override
-        public boolean equals(Object obj) {
-            if (obj == null) {
-                return false;
-            }
-            if (this == obj) {
-                return true;
-            }
-            if (getClass() != obj.getClass()) {
-                return false;
-            }
-            BroadcastEvent other = (BroadcastEvent) obj;
-            if (!StringUtils.equals(action, other.action)) {
-                return false;
-            }
-            if (!StringUtils.equals(name, other.name)) {
-                return false;
-            }
-            if (!StringUtils.equals(type, other.type)) {
-                return false;
-            }
-            return true;
-        }
-
-        @Override
-        public String toString() {
-            return Objects.toStringHelper(this).add("type", type).add("name", name).add("action", action).toString();
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-common/src/main/java/org/apache/kylin/common/restclient/CaseInsensitiveStringCache.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/CaseInsensitiveStringCache.java b/core-common/src/main/java/org/apache/kylin/common/restclient/CaseInsensitiveStringCache.java
deleted file mode 100644
index 2bcddbf..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/CaseInsensitiveStringCache.java
+++ /dev/null
@@ -1,42 +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.common.restclient;
-
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import org.apache.kylin.common.KylinConfig;
-
-/**
- */
-public class CaseInsensitiveStringCache<V> extends SingleValueCache<String, V> {
-
-    public CaseInsensitiveStringCache(KylinConfig config, Broadcaster.TYPE syncType) {
-        super(config, syncType, new ConcurrentSkipListMap<String, V>(String.CASE_INSENSITIVE_ORDER));
-    }
-
-    @Override
-    public void put(String key, V value) {
-        super.put(key, value);
-    }
-
-    @Override
-    public void putLocal(String key, V value) {
-        super.putLocal(key, value);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
index 050d911..46a9e9b 100644
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
+++ b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
@@ -89,8 +89,8 @@ public class RestClient {
         }
     }
 
-    public void wipeCache(String type, String action, String name) throws IOException {
-        String url = baseUrl + "/cache/" + type + "/" + name + "/" + action;
+    public void wipeCache(String entity, String event, String cacheKey) throws IOException {
+        String url = baseUrl + "/cache/" + entity + "/" + cacheKey + "/" + event;
         HttpPut request = new HttpPut(url);
 
         try {

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-common/src/main/java/org/apache/kylin/common/restclient/SingleValueCache.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/SingleValueCache.java b/core-common/src/main/java/org/apache/kylin/common/restclient/SingleValueCache.java
deleted file mode 100644
index 5d1ca9a..0000000
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/SingleValueCache.java
+++ /dev/null
@@ -1,103 +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.common.restclient;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.kylin.common.KylinConfig;
-
-/**
- * @author xjiang
- */
-public abstract class SingleValueCache<K, V> extends AbstractRestCache<K, V> {
-
-    private final ConcurrentMap<K, V> innerCache;
-
-    public SingleValueCache(KylinConfig config, Broadcaster.TYPE syncType) {
-        this(config, syncType, new ConcurrentHashMap<K, V>());
-    }
-
-    public SingleValueCache(KylinConfig config, Broadcaster.TYPE syncType, ConcurrentMap<K, V> innerCache) {
-        super(config, syncType);
-        this.innerCache = innerCache;
-    }
-
-    public void put(K key, V value) {
-        boolean exists = innerCache.containsKey(key);
-
-        innerCache.put(key, value);
-
-        if (!exists) {
-            getBroadcaster().queue(syncType.getType(), Broadcaster.EVENT.CREATE.getType(), key.toString());
-        } else {
-            getBroadcaster().queue(syncType.getType(), Broadcaster.EVENT.UPDATE.getType(), key.toString());
-        }
-    }
-
-    public void putLocal(K key, V value) {
-        innerCache.put(key, value);
-    }
-
-    public void remove(K key) {
-        boolean exists = innerCache.containsKey(key);
-
-        innerCache.remove(key);
-
-        if (exists) {
-            getBroadcaster().queue(syncType.getType(), Broadcaster.EVENT.DROP.getType(), key.toString());
-        }
-    }
-
-    public void removeLocal(K key) {
-        innerCache.remove(key);
-    }
-
-    public void clear() {
-        innerCache.clear();
-    }
-
-    public int size() {
-        return innerCache.size();
-    }
-
-    public V get(K key) {
-        return innerCache.get(key);
-    }
-
-    public Collection<V> values() {
-        return innerCache.values();
-    }
-
-    public boolean containsKey(String key) {
-        return innerCache.containsKey(key);
-    }
-
-    public Map<K, V> getMap() {
-        return Collections.unmodifiableMap(innerCache);
-    }
-
-    public Set<K> keySet() {
-        return innerCache.keySet();
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java b/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
index 556019f..14bf90b 100644
--- a/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
+++ b/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
@@ -49,7 +49,6 @@ public abstract class AbstractKylinTestCase {
         cleanupCache();
         System.clearProperty(KylinConfig.KYLIN_CONF);
         KylinConfig.destroyInstance();
-
     }
 
     private static void cleanupCache() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
index 1b1cf70..d6364fe 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
@@ -27,22 +27,26 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.cube.model.validation.CubeMetadataValidator;
 import org.apache.kylin.cube.model.validation.ValidateContext;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
+import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metadata.project.ProjectManager;
+import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
  * Manager class for CubeDesc; extracted from #CubeManager
+ * 
  * @author shaoshi
- *
  */
 public class CubeDescManager {
 
@@ -90,8 +94,46 @@ public class CubeDescManager {
     private CubeDescManager(KylinConfig config) throws IOException {
         logger.info("Initializing CubeDescManager with config " + config);
         this.config = config;
-        this.cubeDescMap = new CaseInsensitiveStringCache<CubeDesc>(config, Broadcaster.TYPE.CUBE_DESC);
+        this.cubeDescMap = new CaseInsensitiveStringCache<CubeDesc>(config, "cube_desc");
+        
+        // touch lower level metadata before registering my listener
         reloadAllCubeDesc();
+        Broadcaster.getInstance(config).registerListener(new CubeDescSyncListener(), "cube_desc");
+    }
+    
+    private class CubeDescSyncListener extends Broadcaster.Listener {
+        
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+            Cuboid.clearCache();
+        }
+
+        @Override
+        public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException {
+            for (IRealization real : ProjectManager.getInstance(config).listAllRealizations(project)) {
+                if (real instanceof CubeInstance) {
+                    String descName = ((CubeInstance) real).getDescName();
+                    reloadCubeDescLocal(descName);        
+                }
+            }
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            String cubeDescName = cacheKey;
+            CubeDesc cubeDesc = getCubeDesc(cubeDescName);
+            String modelName = cubeDesc == null ? null : cubeDesc.getModel().getName();
+            
+            if (event == Event.DROP)
+                removeLocalCubeDesc(cubeDescName);
+            else
+                reloadCubeDescLocal(cubeDescName);
+            
+            for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByModel(modelName)) {
+                broadcaster.notifyProjectSchemaUpdate(prj.getName());
+            }
+        }
     }
 
     public CubeDesc getCubeDesc(String name) {
@@ -130,6 +172,8 @@ public class CubeDescManager {
     private CubeDesc loadCubeDesc(String path, boolean allowBroken) throws IOException {
         ResourceStore store = getStore();
         CubeDesc ndesc = store.getResource(path, CubeDesc.class, CUBE_DESC_SERIALIZER);
+        if (ndesc == null)
+            throw new IllegalArgumentException("No cube desc found at " + path);
 
         try {
             ndesc.init(config, getMetadataManager().getAllTablesMap());

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
index a2ed051..cad00e1 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeInstance.java
@@ -249,10 +249,6 @@ public class CubeInstance extends RootPersistentEntity implements IRealization,
     }
 
     public String getDescName() {
-        return descName.toUpperCase();
-    }
-
-    public String getOriginDescName() {
         return descName;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
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 fd46b54..2fadedb 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -39,8 +39,6 @@ import org.apache.kylin.common.KylinConfigExt;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
 import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.model.CubeDesc;
@@ -52,9 +50,13 @@ import org.apache.kylin.dict.lookup.LookupStringTable;
 import org.apache.kylin.dict.lookup.SnapshotManager;
 import org.apache.kylin.dict.lookup.SnapshotTable;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
+import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 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.realization.IRealization;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -130,8 +132,41 @@ public class CubeManager implements IRealizationProvider {
     private CubeManager(KylinConfig config) throws IOException {
         logger.info("Initializing CubeManager with config " + config);
         this.config = config;
-        this.cubeMap = new CaseInsensitiveStringCache<CubeInstance>(config, Broadcaster.TYPE.CUBE);
+        this.cubeMap = new CaseInsensitiveStringCache<CubeInstance>(config, "cube");
+        
+        // touch lower level metadata before registering my listener
         loadAllCubeInstance();
+        Broadcaster.getInstance(config).registerListener(new CubeSyncListener(), "cube");
+    }
+
+    private class CubeSyncListener extends Broadcaster.Listener {
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException {
+            for (IRealization real : ProjectManager.getInstance(config).listAllRealizations(project)) {
+                if (real instanceof CubeInstance) {
+                    reloadCubeLocal(real.getName());
+                }
+            }
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            String cubeName = cacheKey;
+            
+            if (event == Event.DROP)
+                removeCubeLocal(cubeName);
+            else
+                reloadCubeLocal(cubeName);
+            
+            for (ProjectInstance prj : ProjectManager.getInstance(config).findProjects(RealizationType.CUBE, cubeName)) {
+                broadcaster.notifyProjectDataUpdate(prj.getName());
+            }
+        }
     }
 
     public List<CubeInstance> listAllCubes() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
index a74dd58..9be3faf 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
@@ -37,9 +37,10 @@ import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
 import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
+import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.ExternalFilterDesc;
@@ -253,15 +254,100 @@ public class MetadataManager {
 
     private void init(KylinConfig config) throws IOException {
         this.config = config;
-        this.srcTableMap = new CaseInsensitiveStringCache<>(config, Broadcaster.TYPE.TABLE);
-        this.srcTableExdMap = new CaseInsensitiveStringCache<>(config, Broadcaster.TYPE.TABLE);
-        this.dataModelDescMap = new CaseInsensitiveStringCache<>(config, Broadcaster.TYPE.DATA_MODEL);
-        this.extFilterMap = new CaseInsensitiveStringCache<>(config, Broadcaster.TYPE.EXTERNAL_FILTER);
+        this.srcTableMap = new CaseInsensitiveStringCache<>(config, "table");
+        this.srcTableExdMap = new CaseInsensitiveStringCache<>(config, "table_ext");
+        this.dataModelDescMap = new CaseInsensitiveStringCache<>(config, "data_model");
+        this.extFilterMap = new CaseInsensitiveStringCache<>(config, "external_filter");
 
         reloadAllSourceTable();
         reloadAllSourceTableExd();
         reloadAllDataModel();
         reloadAllExternalFilter();
+        
+        // touch lower level metadata before registering my listener
+        Broadcaster.getInstance(config).registerListener(new SrcTableSyncListener(), "table");
+        Broadcaster.getInstance(config).registerListener(new SrcTableExtSyncListener(), "table_ext");
+        Broadcaster.getInstance(config).registerListener(new DataModelSyncListener(), "data_model");
+        Broadcaster.getInstance(config).registerListener(new ExtFilterSyncListener(), "external_filter");
+    }
+
+    private class SrcTableSyncListener extends Broadcaster.Listener {
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            if (event == Event.DROP)
+                srcTableMap.removeLocal(cacheKey);
+            else
+                reloadSourceTable(cacheKey);
+            
+            for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByTable(cacheKey)) {
+                broadcaster.notifyProjectSchemaUpdate(prj.getName());
+            }
+        }
+    }
+
+    private class SrcTableExtSyncListener extends Broadcaster.Listener {
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            if (event == Event.DROP)
+                srcTableExdMap.removeLocal(cacheKey);
+            else
+                reloadSourceTableExt(cacheKey);
+            
+            for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByTable(cacheKey)) {
+                broadcaster.notifyProjectSchemaUpdate(prj.getName());
+            }
+        }
+    }
+
+    private class DataModelSyncListener extends Broadcaster.Listener {
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException {
+            for (String model : ProjectManager.getInstance(config).getProject(project).getModels()) {
+                reloadDataModelDesc(model);
+            }
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            if (event == Event.DROP)
+                dataModelDescMap.removeLocal(cacheKey);
+            else
+                reloadDataModelDesc(cacheKey);
+            
+            for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByModel(cacheKey)) {
+                broadcaster.notifyProjectSchemaUpdate(prj.getName());
+            }
+        }
+    }
+
+    private class ExtFilterSyncListener extends Broadcaster.Listener {
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            if (event == Event.DROP)
+                extFilterMap.removeLocal(cacheKey);
+            else
+                reloadExtFilter(cacheKey);
+        }
     }
 
     private void reloadAllSourceTableExd() throws IOException {
@@ -454,8 +540,8 @@ public class MetadataManager {
             dataModelDesc.init(config, this.getAllTablesMap());
             dataModelDescMap.putLocal(dataModelDesc.getName(), dataModelDesc);
             return dataModelDesc;
-        } catch (IOException e) {
-            throw new IllegalStateException("Error to load" + path, e);
+        } catch (Exception e) {
+            throw new IllegalStateException("Error to load " + path, e);
         }
     }
 
@@ -484,9 +570,10 @@ public class MetadataManager {
         String name = desc.getName();
         if (dataModelDescMap.containsKey(name))
             throw new IllegalArgumentException("DataModelDesc '" + name + "' already exists");
-        ProjectManager.getInstance(config).updateModelToProject(name, projectName);
         desc.setOwner(owner);
-        return saveDataModelDesc(desc);
+        desc = saveDataModelDesc(desc);
+        ProjectManager.getInstance(config).updateModelToProject(name, projectName);
+        return desc;
     }
 
     public DataModelDesc updateDataModelDesc(DataModelDesc desc) throws IOException {

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/AbstractCache.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/AbstractCache.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/AbstractCache.java
new file mode 100644
index 0000000..4894817
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/AbstractCache.java
@@ -0,0 +1,50 @@
+/*
+ * 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.metadata.cachesync;
+
+import org.apache.kylin.common.KylinConfig;
+
+/**
+ */
+public abstract class AbstractCache<K, V> {
+
+    protected final KylinConfig config;
+    protected final String syncEntity;
+
+    protected AbstractCache(KylinConfig config, String syncEntity) {
+        this.config = config;
+        this.syncEntity = syncEntity;
+    }
+
+    public Broadcaster getBroadcaster() {
+        return Broadcaster.getInstance(config);
+    }
+
+    public abstract void put(K key, V value);
+
+    public abstract void putLocal(K key, V value);
+
+    public abstract void remove(K key);
+
+    public abstract void removeLocal(K key);
+
+    public abstract void clear();
+
+    public abstract int size();
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
new file mode 100644
index 0000000..73dd0a7
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
@@ -0,0 +1,332 @@
+/*
+ * 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.metadata.cachesync;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.restclient.RestClient;
+import org.apache.kylin.common.util.DaemonThreadFactory;
+import org.apache.kylin.metadata.project.ProjectManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * Broadcast kylin event out
+ */
+public class Broadcaster {
+
+    private static final Logger logger = LoggerFactory.getLogger(Broadcaster.class);
+
+    public static final String SYNC_ALL = "all"; // the special entity to indicate clear all
+    public static final String SYNC_PRJ_SCHEMA = "project_schema"; // the special entity to indicate project schema has change, e.g. table/model/cube_desc update
+    public static final String SYNC_PRJ_DATA = "project_data"; // the special entity to indicate project data has change, e.g. cube/raw_table update
+
+    // static cached instances
+    private static final ConcurrentHashMap<KylinConfig, Broadcaster> CACHE = new ConcurrentHashMap<KylinConfig, Broadcaster>();
+
+    public static Broadcaster getInstance(KylinConfig config) {
+        Broadcaster r = CACHE.get(config);
+        if (r != null) {
+            return r;
+        }
+
+        synchronized (Broadcaster.class) {
+            r = CACHE.get(config);
+            if (r != null) {
+                return r;
+            }
+
+            r = new Broadcaster(config);
+            CACHE.put(config, r);
+            if (CACHE.size() > 1) {
+                logger.warn("More than one singleton exist");
+            }
+            return r;
+        }
+    }
+
+    public static void clearCache() {
+        CACHE.clear();
+    }
+
+    // ============================================================================
+
+    private KylinConfig config;
+
+    private BlockingDeque<BroadcastEvent> broadcastEvents = new LinkedBlockingDeque<>();
+    private Map<String, List<Listener>> listenerMap = Maps.newConcurrentMap();
+    private AtomicLong counter = new AtomicLong();
+
+    private Broadcaster(final KylinConfig config) {
+        this.config = config;
+
+        final String[] nodes = config.getRestServers();
+        if (nodes == null || nodes.length < 1) {
+            logger.warn("There is no available rest server; check the 'kylin.rest.servers' config");
+            broadcastEvents = null; // disable the broadcaster
+            return;
+        }
+        logger.debug(nodes.length + " nodes in the cluster: " + Arrays.toString(nodes));
+
+        Executors.newSingleThreadExecutor(new DaemonThreadFactory()).execute(new Runnable() {
+            @Override
+            public void run() {
+                final List<RestClient> restClients = Lists.newArrayList();
+                for (String node : nodes) {
+                    restClients.add(new RestClient(node));
+                }
+                final ExecutorService wipingCachePool = Executors.newFixedThreadPool(restClients.size());
+                while (true) {
+                    try {
+                        final BroadcastEvent broadcastEvent = broadcastEvents.takeFirst();
+                        logger.info("Announcing new broadcast event:" + broadcastEvent);
+                        for (final RestClient restClient : restClients) {
+                            wipingCachePool.execute(new Runnable() {
+                                @Override
+                                public void run() {
+                                    try {
+                                        restClient.wipeCache(broadcastEvent.getEntity(), broadcastEvent.getEvent(), broadcastEvent.getCacheKey());
+                                    } catch (IOException e) {
+                                        logger.warn("Thread failed during wipe cache at " + broadcastEvent);
+                                    }
+                                }
+                            });
+                        }
+                    } catch (Exception e) {
+                        logger.error("error running wiping", e);
+                    }
+                }
+            }
+        });
+    }
+
+    public void registerListener(Listener listener, String... entities) {
+        // ignore re-registration
+        List<Listener> all = listenerMap.get(SYNC_ALL);
+        if (all != null && all.contains(listener)) {
+            return;
+        }
+
+        for (String entity : entities) {
+            if (!StringUtils.isBlank(entity))
+                addListener(entity, listener);
+        }
+        addListener(SYNC_ALL, listener);
+        addListener(SYNC_PRJ_SCHEMA, listener);
+        addListener(SYNC_PRJ_DATA, listener);
+    }
+
+    synchronized private void addListener(String entity, Listener listener) {
+        List<Listener> list = listenerMap.get(entity);
+        if (list == null) {
+            list = new ArrayList<>();
+        }
+        list.add(listener);
+        listenerMap.put(entity, list);
+    }
+
+    public void notifyClearAll() throws IOException {
+        notifyListener(SYNC_ALL, Event.UPDATE, SYNC_ALL);
+    }
+
+    public void notifyProjectSchemaUpdate(String project) throws IOException {
+        notifyListener(SYNC_PRJ_SCHEMA, Event.UPDATE, project);
+    }
+
+    public void notifyProjectDataUpdate(String project) throws IOException {
+        notifyListener(SYNC_PRJ_DATA, Event.UPDATE, project);
+    }
+
+    public synchronized void notifyListener(String entity, Event event, String cacheKey) throws IOException {
+        List<Listener> list = listenerMap.get(entity);
+        if (list == null)
+            return;
+        
+        logger.debug("Broadcasting metadata change: entity=" + entity + ", event=" + event + ", cacheKey=" + cacheKey + ", listeners=" + list);
+        
+        // prevents concurrent modification exception
+        list = Lists.newArrayList(list);
+        switch (entity) {
+        case SYNC_ALL:
+            for (Listener l : list) {
+                l.onClearAll(this);
+            }
+            clearCache(); // clear broadcaster too in the end
+            break;
+        case SYNC_PRJ_SCHEMA:
+            ProjectManager.getInstance(config).clearL2Cache();
+            for (Listener l : list) {
+                l.onProjectSchemaChange(this, cacheKey);
+            }
+            break;
+        case SYNC_PRJ_DATA:
+            ProjectManager.getInstance(config).clearL2Cache(); // cube's first becoming ready leads to schema change too
+            for (Listener l : list) {
+                l.onProjectDataChange(this, cacheKey);
+            }
+            break;
+        default:
+            for (Listener l : list) {
+                l.onEntityChange(this, entity, event, cacheKey);
+            }
+            break;
+        }
+        
+        logger.debug("Done broadcasting metadata change: entity=" + entity + ", event=" + event + ", cacheKey=" + cacheKey);
+    }
+
+    /**
+     * Broadcast an event out
+     */
+    public void queue(String entity, String event, String key) {
+        if (broadcastEvents == null)
+            return;
+
+        try {
+            counter.incrementAndGet();
+            broadcastEvents.putFirst(new BroadcastEvent(entity, event, key));
+        } catch (Exception e) {
+            counter.decrementAndGet();
+            logger.error("error putting BroadcastEvent", e);
+        }
+    }
+
+    public long getCounterAndClear() {
+        return counter.getAndSet(0);
+    }
+
+    public enum Event {
+
+        CREATE("create"), UPDATE("update"), DROP("drop");
+        private String text;
+
+        Event(String text) {
+            this.text = text;
+        }
+
+        public String getType() {
+            return text;
+        }
+
+        public static Event getEvent(String event) {
+            for (Event one : values()) {
+                if (one.getType().equalsIgnoreCase(event)) {
+                    return one;
+                }
+            }
+
+            return null;
+        }
+    }
+
+    abstract public static class Listener {
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+        }
+
+        public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException {
+        }
+
+        public void onProjectDataChange(Broadcaster broadcaster, String project) throws IOException {
+        }
+
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+        }
+    }
+
+    public static class BroadcastEvent {
+        private String entity;
+        private String event;
+        private String cacheKey;
+
+        public BroadcastEvent(String entity, String event, String cacheKey) {
+            super();
+            this.entity = entity;
+            this.event = event;
+            this.cacheKey = cacheKey;
+        }
+
+        public String getEntity() {
+            return entity;
+        }
+
+        public String getEvent() {
+            return event;
+        }
+
+        public String getCacheKey() {
+            return cacheKey;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((event == null) ? 0 : event.hashCode());
+            result = prime * result + ((cacheKey == null) ? 0 : cacheKey.hashCode());
+            result = prime * result + ((entity == null) ? 0 : entity.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == null) {
+                return false;
+            }
+            if (this == obj) {
+                return true;
+            }
+            if (getClass() != obj.getClass()) {
+                return false;
+            }
+            BroadcastEvent other = (BroadcastEvent) obj;
+            if (!StringUtils.equals(event, other.event)) {
+                return false;
+            }
+            if (!StringUtils.equals(cacheKey, other.cacheKey)) {
+                return false;
+            }
+            if (!StringUtils.equals(entity, other.entity)) {
+                return false;
+            }
+            return true;
+        }
+
+        @Override
+        public String toString() {
+            return Objects.toStringHelper(this).add("type", entity).add("name", cacheKey).add("action", event).toString();
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/CaseInsensitiveStringCache.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/CaseInsensitiveStringCache.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/CaseInsensitiveStringCache.java
new file mode 100644
index 0000000..b4d0438
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/CaseInsensitiveStringCache.java
@@ -0,0 +1,42 @@
+/*
+ * 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.metadata.cachesync;
+
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.kylin.common.KylinConfig;
+
+/**
+ */
+public class CaseInsensitiveStringCache<V> extends SingleValueCache<String, V> {
+
+    public CaseInsensitiveStringCache(KylinConfig config, String syncEntity) {
+        super(config, syncEntity, new ConcurrentSkipListMap<String, V>(String.CASE_INSENSITIVE_ORDER));
+    }
+
+    @Override
+    public void put(String key, V value) {
+        super.put(key, value);
+    }
+
+    @Override
+    public void putLocal(String key, V value) {
+        super.putLocal(key, value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/SingleValueCache.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/SingleValueCache.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/SingleValueCache.java
new file mode 100644
index 0000000..4bfaeae
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/SingleValueCache.java
@@ -0,0 +1,103 @@
+/*
+ * 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.metadata.cachesync;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.kylin.common.KylinConfig;
+
+/**
+ * @author xjiang
+ */
+public abstract class SingleValueCache<K, V> extends AbstractCache<K, V> {
+
+    private final ConcurrentMap<K, V> innerCache;
+
+    public SingleValueCache(KylinConfig config, String syncEntity) {
+        this(config, syncEntity, new ConcurrentHashMap<K, V>());
+    }
+
+    public SingleValueCache(KylinConfig config, String syncEntity, ConcurrentMap<K, V> innerCache) {
+        super(config, syncEntity);
+        this.innerCache = innerCache;
+    }
+
+    public void put(K key, V value) {
+        boolean exists = innerCache.containsKey(key);
+
+        innerCache.put(key, value);
+
+        if (!exists) {
+            getBroadcaster().queue(syncEntity, Broadcaster.Event.CREATE.getType(), key.toString());
+        } else {
+            getBroadcaster().queue(syncEntity, Broadcaster.Event.UPDATE.getType(), key.toString());
+        }
+    }
+
+    public void putLocal(K key, V value) {
+        innerCache.put(key, value);
+    }
+
+    public void remove(K key) {
+        boolean exists = innerCache.containsKey(key);
+
+        innerCache.remove(key);
+
+        if (exists) {
+            getBroadcaster().queue(syncEntity, Broadcaster.Event.DROP.getType(), key.toString());
+        }
+    }
+
+    public void removeLocal(K key) {
+        innerCache.remove(key);
+    }
+
+    public void clear() {
+        innerCache.clear();
+    }
+
+    public int size() {
+        return innerCache.size();
+    }
+
+    public V get(K key) {
+        return innerCache.get(key);
+    }
+
+    public Collection<V> values() {
+        return innerCache.values();
+    }
+
+    public boolean containsKey(String key) {
+        return innerCache.containsKey(key);
+    }
+
+    public Map<K, V> getMap() {
+        return Collections.unmodifiableMap(innerCache);
+    }
+
+    public Set<K> keySet() {
+        return innerCache.keySet();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java
index 1bf9804..2838e56 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/project/ProjectManager.java
@@ -29,10 +29,11 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.badquery.BadQueryHistoryManager;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
+import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.ExternalFilterDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
@@ -87,10 +88,33 @@ public class ProjectManager {
     private ProjectManager(KylinConfig config) throws IOException {
         logger.info("Initializing ProjectManager with metadata url " + config);
         this.config = config;
-        this.projectMap = new CaseInsensitiveStringCache<ProjectInstance>(config, Broadcaster.TYPE.PROJECT);
+        this.projectMap = new CaseInsensitiveStringCache<ProjectInstance>(config, "project");
         this.l2Cache = new ProjectL2Cache(this);
 
+        // touch lower level metadata before registering my listener
         reloadAllProjects();
+        Broadcaster.getInstance(config).registerListener(new ProjectSyncListener(), "project");
+    }
+
+    private class ProjectSyncListener extends Broadcaster.Listener {
+        
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            String project = cacheKey;
+            
+            if (event == Event.DROP)
+                removeProjectLocal(project);
+            else
+                reloadProjectLocal(project);
+            
+            broadcaster.notifyProjectSchemaUpdate(project);
+            broadcaster.notifyProjectDataUpdate(project);
+        }
     }
 
     public void clearL2Cache() {
@@ -224,6 +248,11 @@ public class ProjectManager {
         projectMap.remove(norm(proj.getName()));
         clearL2Cache();
     }
+    
+    private void removeProjectLocal(String proj) {
+        projectMap.remove(norm(proj));
+        clearL2Cache();
+    }
 
     public boolean isModelInProject(String projectName, String modelName) {
         return this.getProject(projectName).containsModel(modelName);
@@ -235,7 +264,7 @@ public class ProjectManager {
     }
 
     public void removeModelFromProjects(String modelName) throws IOException {
-        for (ProjectInstance projectInstance : findProjects(modelName)) {
+        for (ProjectInstance projectInstance : findProjectsByModel(modelName)) {
             projectInstance.removeModel(modelName);
             updateProject(projectInstance);
         }
@@ -344,17 +373,26 @@ public class ProjectManager {
         return result;
     }
 
-    private List<ProjectInstance> findProjects(String modelName) {
+    public List<ProjectInstance> findProjectsByModel(String modelName) {
         List<ProjectInstance> projects = new ArrayList<ProjectInstance>();
         for (ProjectInstance projectInstance : projectMap.values()) {
             if (projectInstance.containsModel(modelName)) {
                 projects.add(projectInstance);
             }
         }
-
         return projects;
     }
 
+    public List<ProjectInstance> findProjectsByTable(String tableIdentity) {
+        List<ProjectInstance> projects = new ArrayList<ProjectInstance>();
+        for (ProjectInstance projectInstance : projectMap.values()) {
+            if (projectInstance.containsTable(tableIdentity)) {
+                projects.add(projectInstance);
+            }
+        }
+        return projects;
+    }
+    
     public ExternalFilterDesc getExternalFilterDesc(String project, String extFilter) {
         return l2Cache.getExternalFilterDesc(project, extFilter);
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java
index 0f948cb..4f81b09 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/hybrid/HybridManager.java
@@ -27,8 +27,11 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
+import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metadata.project.ProjectManager;
 import org.apache.kylin.metadata.project.RealizationEntry;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.metadata.realization.IRealizationProvider;
@@ -83,18 +86,52 @@ public class HybridManager implements IRealizationProvider {
     private HybridManager(KylinConfig config) throws IOException {
         logger.info("Initializing HybridManager with config " + config);
         this.config = config;
-        this.hybridMap = new CaseInsensitiveStringCache<HybridInstance>(config, Broadcaster.TYPE.HYBRID);
-        loadAllHybridInstance();
+        this.hybridMap = new CaseInsensitiveStringCache<HybridInstance>(config, "hybrid");
+        
+        // touch lower level metadata before registering my listener
+        reloadAllHybridInstance();
+        Broadcaster.getInstance(config).registerListener(new HybridSyncListener(), "hybrid");
     }
 
-    private void loadAllHybridInstance() throws IOException {
+    private class HybridSyncListener extends Broadcaster.Listener {
+        
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException {
+            for (IRealization real : ProjectManager.getInstance(config).listAllRealizations(project)) {
+                if (real instanceof HybridInstance) {
+                    reloadHybridInstance(real.getName());
+                }
+            }
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            String hybridName = cacheKey;
+            
+            if (event == Event.DROP)
+                hybridMap.removeLocal(hybridName);
+            else
+                reloadHybridInstance(hybridName);
+            
+            for (ProjectInstance prj : ProjectManager.getInstance(config).findProjects(RealizationType.HYBRID, hybridName)) {
+                broadcaster.notifyProjectSchemaUpdate(prj.getName());
+            }
+        }
+    }
+
+    private void reloadAllHybridInstance() throws IOException {
         ResourceStore store = getStore();
         List<String> paths = store.collectResourceRecursively(ResourceStore.HYBRID_RESOURCE_ROOT, ".json");
 
         logger.debug("Loading Hybrid from folder " + store.getReadableResourcePath(ResourceStore.HYBRID_RESOURCE_ROOT));
 
         for (String path : paths) {
-            loadHybridInstance(path);
+            reloadHybridInstanceAt(path);
         }
 
         logger.debug("Loaded " + paths.size() + " Hybrid(s)");
@@ -111,11 +148,15 @@ public class HybridManager implements IRealizationProvider {
             }
 
             if (includes == true)
-                loadHybridInstance(HybridInstance.concatResourcePath(hybridInstance.getName()));
+                reloadHybridInstance(hybridInstance.getName());
         }
     }
 
-    private synchronized HybridInstance loadHybridInstance(String path) {
+    public void reloadHybridInstance(String name) {
+        reloadHybridInstanceAt(HybridInstance.concatResourcePath(name));
+    }
+    
+    private synchronized HybridInstance reloadHybridInstanceAt(String path) {
         ResourceStore store = getStore();
 
         HybridInstance hybridInstance = null;

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java
index e4e1359..7bf6ca2 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java
@@ -44,9 +44,10 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.CaseInsensitiveStringCache;
 import org.apache.kylin.metadata.MetadataConstants;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
+import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,8 +73,26 @@ public class StreamingManager {
 
     private StreamingManager(KylinConfig config) throws IOException {
         this.config = config;
-        this.streamingMap = new CaseInsensitiveStringCache<StreamingConfig>(config, Broadcaster.TYPE.STREAMING);
+        this.streamingMap = new CaseInsensitiveStringCache<StreamingConfig>(config, "streaming");
+        
+        // touch lower level metadata before registering my listener
         reloadAllStreaming();
+        Broadcaster.getInstance(config).registerListener(new StreamingSyncListener(), "streaming");
+    }
+
+    private class StreamingSyncListener extends Broadcaster.Listener {
+        @Override
+        public void onClearAll(Broadcaster broadcaster) throws IOException {
+            clearCache();
+        }
+
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            if (event == Event.DROP)
+                removeStreamingLocal(cacheKey);
+            else
+                reloadStreamingConfigLocal(cacheKey);
+        }
     }
 
     private ResourceStore getStore() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
index 845ffe0..667046b 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
@@ -20,8 +20,7 @@ package org.apache.kylin.rest.controller;
 
 import java.io.IOException;
 
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.common.restclient.Broadcaster.EVENT;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.apache.kylin.rest.service.CacheService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,6 +39,8 @@ import org.springframework.web.bind.annotation.ResponseBody;
 @Controller
 @RequestMapping(value = "/cache")
 public class CacheController extends BasicController {
+    
+    @SuppressWarnings("unused")
     private static final Logger logger = LoggerFactory.getLogger(CacheController.class);
 
     @Autowired
@@ -48,32 +49,16 @@ public class CacheController extends BasicController {
     /**
      * Wipe system cache
      *
-     * @param type  {@link Broadcaster.TYPE}
-     * @param event {@link Broadcaster.EVENT}
-     * @param name
+     * @param entity  {@link Broadcaster.TYPE}
+     * @param event {@link Broadcaster.Event}
+     * @param cacheKey
      * @return if the action success
      * @throws IOException
      */
-    @RequestMapping(value = "/{type}/{name}/{event}", method = { RequestMethod.PUT })
+    @RequestMapping(value = "/{entity}/{cacheKey}/{event}", method = { RequestMethod.PUT })
     @ResponseBody
-    public void wipeCache(@PathVariable String type, @PathVariable String event, @PathVariable String name) throws IOException {
-
-        Broadcaster.TYPE wipeType = Broadcaster.TYPE.getType(type);
-        EVENT wipeEvent = Broadcaster.EVENT.getEvent(event);
-
-        logger.info("wipe cache type: " + wipeType + " event:" + wipeEvent + " name:" + name);
-
-        switch (wipeEvent) {
-        case CREATE:
-        case UPDATE:
-            cacheService.rebuildCache(wipeType, name);
-            break;
-        case DROP:
-            cacheService.removeCache(wipeType, name);
-            break;
-        default:
-            throw new RuntimeException("invalid type:" + wipeEvent);
-        }
+    public void wipeCache(@PathVariable String entity, @PathVariable String event, @PathVariable String cacheKey) throws IOException {
+        cacheService.notifyMetadataChange(entity, Broadcaster.Event.getEvent(event), cacheKey);
     }
 
     public void setCacheService(CacheService cacheService) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
index 9d134d6..5d29dcd 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CacheService.java
@@ -21,36 +21,22 @@ package org.apache.kylin.rest.service;
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.Charset;
-import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import javax.annotation.PostConstruct;
 import javax.sql.DataSource;
 
 import org.apache.calcite.jdbc.Driver;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.restclient.Broadcaster;
-import org.apache.kylin.cube.CubeDescManager;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.dict.DictionaryManager;
-import org.apache.kylin.engine.streaming.StreamingManager;
-import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
+import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
 import org.apache.kylin.metadata.project.ProjectInstance;
-import org.apache.kylin.metadata.project.ProjectManager;
-import org.apache.kylin.metadata.realization.RealizationRegistry;
-import org.apache.kylin.metadata.realization.RealizationType;
 import org.apache.kylin.query.enumerator.OLAPQuery;
 import org.apache.kylin.query.schema.OLAPSchemaFactory;
 import org.apache.kylin.rest.controller.QueryController;
-import org.apache.kylin.source.kafka.KafkaConfigManager;
-import org.apache.kylin.storage.hbase.HBaseConnection;
-import org.apache.kylin.storage.hybrid.HybridManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -66,48 +52,64 @@ public class CacheService extends BasicService {
 
     private static final Logger logger = LoggerFactory.getLogger(CacheService.class);
 
-    private static ConcurrentMap<String, DataSource> olapDataSources = new ConcurrentHashMap<String, DataSource>();
+    private ConcurrentMap<String, DataSource> olapDataSources = new ConcurrentHashMap<String, DataSource>();
 
     @Autowired
     private CubeService cubeService;
 
     @Autowired
     private CacheManager cacheManager;
+    
+    private Broadcaster.Listener cacheSyncListener = new Broadcaster.Listener() {
+        @Override
+        public void onProjectSchemaChange(Broadcaster broadcaster, String project) throws IOException {
+            removeOLAPDataSource(project);
+        }
 
-    @PostConstruct
-    public void initCubeChangeListener() throws IOException {
-        CubeManager cubeMgr = CubeManager.getInstance(getConfig());
-        cubeMgr.setCubeChangeListener(new CubeManager.CubeChangeListener() {
-
-            @Override
-            public void afterCubeCreate(CubeInstance cube) {
-                // no cache need change
-            }
-
-            @Override
-            public void afterCubeUpdate(CubeInstance cube) {
-                rebuildCubeCache(cube.getName());
-            }
+        @Override
+        public void onProjectDataChange(Broadcaster broadcaster, String project) throws IOException {
+            cleanDataCache(project);
+        }
 
-            @Override
-            public void afterCubeDelete(CubeInstance cube) {
-                removeCubeCache(cube.getName(), cube);
+        @Override
+        public void onEntityChange(Broadcaster broadcaster, String entity, Event event, String cacheKey) throws IOException {
+            if ("cube".equals(entity) && event == Event.UPDATE) {
+                final String cubeName = cacheKey;
+                new Thread() { // do not block the event broadcast thread
+                    public void run() {
+                        try {
+                            Thread.sleep(1000);
+                            cubeService.updateOnNewSegmentReady(cubeName);
+                        } catch (Throwable ex) {
+                            logger.error("Error in updateOnNewSegmentReady()", ex);
+                        }
+                    }
+                }.run();
             }
-        });
-    }
+        }
+    };
 
     // for test
     public void setCubeService(CubeService cubeService) {
         this.cubeService = cubeService;
     }
 
-    protected void cleanDataCache(String storageUUID) {
+    public void notifyMetadataChange(String entity, Event event, String cacheKey) throws IOException {
+        Broadcaster broadcaster = Broadcaster.getInstance(getConfig());
+        
+        // broadcaster can be clearCache() too, make sure listener is registered; re-registration will be ignored
+        broadcaster.registerListener(cacheSyncListener, "cube");
+        
+        broadcaster.notifyListener(entity, event, cacheKey);
+    }
+
+    protected void cleanDataCache(String project) {
         if (cacheManager != null) {
-            logger.info("cleaning cache for " + storageUUID + " (currently remove all entries)");
+            logger.info("cleaning cache for project" + project + " (currently remove all entries)");
             cacheManager.getCache(QueryController.SUCCESS_QUERY_CACHE).removeAll();
             cacheManager.getCache(QueryController.EXCEPTION_QUERY_CACHE).removeAll();
         } else {
-            logger.warn("skip cleaning cache for " + storageUUID);
+            logger.warn("skip cleaning cache for project " + project);
         }
     }
 
@@ -120,7 +122,7 @@ public class CacheService extends BasicService {
         }
     }
 
-    private static void removeOLAPDataSource(String project) {
+    private void removeOLAPDataSource(String project) {
         logger.info("removeOLAPDataSource is called for project " + project);
         if (StringUtils.isEmpty(project))
             throw new IllegalArgumentException("removeOLAPDataSource: project name not given");
@@ -129,7 +131,7 @@ public class CacheService extends BasicService {
         olapDataSources.remove(project);
     }
 
-    public static void removeAllOLAPDataSources() {
+    public void removeAllOLAPDataSources() {
         // brutal, yet simplest way
         logger.info("removeAllOLAPDataSources is called.");
         olapDataSources.clear();
@@ -166,134 +168,4 @@ public class CacheService extends BasicService {
         return ret;
     }
 
-    public void rebuildCache(Broadcaster.TYPE cacheType, String cacheKey) {
-        final String log = "rebuild cache type: " + cacheType + " name:" + cacheKey;
-        logger.info(log);
-        try {
-            switch (cacheType) {
-            case CUBE:
-                rebuildCubeCache(cacheKey);
-                break;
-            case STREAMING:
-                getStreamingManager().reloadStreamingConfigLocal(cacheKey);
-                break;
-            case KAFKA:
-                getKafkaManager().reloadKafkaConfigLocal(cacheKey);
-                break;
-            case CUBE_DESC:
-                getCubeDescManager().reloadCubeDescLocal(cacheKey);
-                break;
-            case PROJECT:
-                reloadProjectCache(cacheKey);
-                break;
-            case TABLE:
-                getMetadataManager().reloadTableCache(cacheKey);
-                CubeDescManager.clearCache();
-                clearRealizationCache();
-                break;
-            case EXTERNAL_FILTER:
-                getMetadataManager().reloadExtFilter(cacheKey);
-                CubeDescManager.clearCache();
-                break;
-            case DATA_MODEL:
-                getMetadataManager().reloadDataModelDesc(cacheKey);
-                CubeDescManager.clearCache();
-                break;
-            case ALL:
-                DictionaryManager.clearCache();
-                MetadataManager.clearCache();
-                CubeDescManager.clearCache();
-                clearRealizationCache();
-                Cuboid.clearCache();
-                ProjectManager.clearCache();
-                KafkaConfigManager.clearCache();
-                StreamingManager.clearCache();
-                HBaseConnection.clearConnCache();
-
-                cleanAllDataCache();
-                removeAllOLAPDataSources();
-                break;
-            default:
-                logger.error("invalid cacheType:" + cacheType);
-            }
-        } catch (IOException e) {
-            throw new RuntimeException("error " + log, e);
-        }
-    }
-
-    private void clearRealizationCache() {
-        CubeManager.clearCache();
-        HybridManager.clearCache();
-        RealizationRegistry.clearCache();
-    }
-
-    private void rebuildCubeCache(String cubeName) {
-        CubeInstance cube = getCubeManager().reloadCubeLocal(cubeName);
-        getHybridManager().reloadHybridInstanceByChild(RealizationType.CUBE, cubeName);
-        reloadProjectCache(getProjectManager().findProjects(RealizationType.CUBE, cubeName));
-        //clean query related cache first
-        if (cube != null) {
-            cleanDataCache(cube.getUuid());
-        }
-        cubeService.updateOnNewSegmentReady(cubeName);
-    }
-
-    public void removeCache(Broadcaster.TYPE cacheType, String cacheKey) {
-        final String log = "remove cache type: " + cacheType + " name:" + cacheKey;
-        try {
-            switch (cacheType) {
-            case CUBE:
-                removeCubeCache(cacheKey, null);
-                break;
-            case CUBE_DESC:
-                getCubeDescManager().removeLocalCubeDesc(cacheKey);
-                break;
-            case PROJECT:
-                ProjectManager.clearCache();
-                break;
-            case TABLE:
-                throw new UnsupportedOperationException(log);
-            case EXTERNAL_FILTER:
-                throw new UnsupportedOperationException(log);
-            case DATA_MODEL:
-                getMetadataManager().removeModelCache(cacheKey);
-                break;
-            default:
-                throw new RuntimeException("invalid cacheType:" + cacheType);
-            }
-        } catch (IOException e) {
-            throw new RuntimeException("error " + log, e);
-        }
-    }
-
-    private void removeCubeCache(String cubeName, CubeInstance cube) {
-        // you may not get the cube instance if it's already removed from metadata
-        if (cube == null) {
-            cube = getCubeManager().getCube(cubeName);
-        }
-
-        getCubeManager().removeCubeLocal(cubeName);
-        getHybridManager().reloadHybridInstanceByChild(RealizationType.CUBE, cubeName);
-        reloadProjectCache(getProjectManager().findProjects(RealizationType.CUBE, cubeName));
-
-        if (cube != null) {
-            cleanDataCache(cube.getUuid());
-        }
-    }
-
-    private void reloadProjectCache(List<ProjectInstance> projects) {
-        for (ProjectInstance prj : projects) {
-            reloadProjectCache(prj.getName());
-        }
-    }
-
-    private void reloadProjectCache(String projectName) {
-        try {
-            getProjectManager().reloadProjectLocal(projectName);
-        } catch (IOException ex) {
-            logger.warn("Failed to reset project cache", ex);
-        }
-        removeOLAPDataSource(projectName);
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
index 38cd93f..af9ccc0 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
@@ -22,13 +22,13 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.restclient.Broadcaster;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
 import org.apache.kylin.cube.CubeDescManager;
 import org.apache.kylin.cube.CubeInstance;
@@ -36,6 +36,7 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.metadata.MetadataManager;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.LookupDesc;
 import org.apache.kylin.metadata.model.TableDesc;
@@ -109,32 +110,19 @@ public class CacheServiceTest extends LocalFileMetadataTestCase {
         };
 
         serviceA.setCubeService(cubeServiceA);
-        serviceA.initCubeChangeListener();
         serviceB.setCubeService(cubeServiceB);
-        serviceB.initCubeChangeListener();
 
         context.addServlet(new ServletHolder(new BroadcasterReceiveServlet(new BroadcasterReceiveServlet.BroadcasterHandler() {
             @Override
-            public void handle(String type, String name, String event) {
-
-                Broadcaster.TYPE wipeType = Broadcaster.TYPE.getType(type);
-                Broadcaster.EVENT wipeEvent = Broadcaster.EVENT.getEvent(event);
-                final String log = "wipe cache type: " + wipeType + " event:" + wipeEvent + " name:" + name;
+            public void handle(String entity, String cacheKey, String event) {
+                Broadcaster.Event wipeEvent = Broadcaster.Event.getEvent(event);
+                final String log = "wipe cache type: " + entity + " event:" + wipeEvent + " name:" + cacheKey;
                 logger.info(log);
                 try {
-                    switch (wipeEvent) {
-                    case CREATE:
-                    case UPDATE:
-                        serviceA.rebuildCache(wipeType, name);
-                        serviceB.rebuildCache(wipeType, name);
-                        break;
-                    case DROP:
-                        serviceA.removeCache(wipeType, name);
-                        serviceB.removeCache(wipeType, name);
-                        break;
-                    default:
-                        throw new RuntimeException("invalid type:" + wipeEvent);
-                    }
+                    serviceA.notifyMetadataChange(entity, wipeEvent, cacheKey);
+                    serviceB.notifyMetadataChange(entity, wipeEvent, cacheKey);
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
                 } finally {
                     counter.incrementAndGet();
                 }
@@ -153,12 +141,10 @@ public class CacheServiceTest extends LocalFileMetadataTestCase {
     @Before
     public void setUp() throws Exception {
         counter.set(0L);
-        createTestMetadata();
     }
 
     @After
     public void after() throws Exception {
-        cleanupTestMetadata();
     }
 
     private void waitForCounterAndClear(long count) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/72005ea5/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java b/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java
index ae4c089..3a587e4 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/ServiceTestBase.java
@@ -18,12 +18,9 @@
 
 package org.apache.kylin.rest.service;
 
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.cube.CubeDescManager;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.project.ProjectManager;
-import org.apache.kylin.metadata.realization.RealizationRegistry;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -60,12 +57,8 @@ public class ServiceTestBase extends LocalFileMetadataTestCase {
     public void setup() throws Exception {
         this.createTestMetadata();
 
-        MetadataManager.clearCache();
-        CubeDescManager.clearCache();
-        CubeManager.clearCache();
-        RealizationRegistry.clearCache();
-        ProjectManager.clearCache();
-        CacheService.removeAllOLAPDataSources();
+        KylinConfig config = KylinConfig.getInstanceFromEnv();
+        Broadcaster.getInstance(config).notifyClearAll();
     }
 
     @After


[32/50] [abbrv] kylin git commit: KYLIN-2044 Fix unclosed DataInputByteBuffer in BitmapCounter#peekLength

Posted by li...@apache.org.
KYLIN-2044 Fix unclosed DataInputByteBuffer in BitmapCounter#peekLength


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: a346db5108b55acb667539a5269d1b1864ea3084
Parents: 818188d
Author: lidongsjtu <li...@apache.org>
Authored: Sat Sep 24 10:30:40 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Sat Sep 24 10:42:46 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/measure/bitmap/BitmapCounter.java   | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a346db51/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
index b52adde..d3b57a7 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
@@ -26,6 +26,7 @@ import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
 
+import org.apache.commons.io.IOUtils;
 import org.roaringbitmap.buffer.MutableRoaringBitmap;
 
 /**
@@ -97,8 +98,12 @@ public class BitmapCounter implements Comparable<BitmapCounter> {
 
     public void readRegisters(ByteBuffer in) throws IOException {
         DataInputByteBuffer input = new DataInputByteBuffer();
-        input.reset(new ByteBuffer[] { in });
-        bitmap.deserialize(input);
+        try {
+            input.reset(new ByteBuffer[] { in });
+            bitmap.deserialize(input);
+        } finally {
+            IOUtils.closeQuietly(input);
+        }
     }
 
     @Override
@@ -148,6 +153,8 @@ public class BitmapCounter implements Comparable<BitmapCounter> {
             bitmap.deserialize(input);
         } catch (IOException e) {
             throw new IllegalStateException(e);
+        } finally {
+            IOUtils.closeQuietly(input);
         }
 
         len = in.position() - mark;


[40/50] [abbrv] kylin git commit: KYLIN-1999: Fix UT for the config update

Posted by li...@apache.org.
KYLIN-1999: Fix UT for the config update

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: b95e9671042240761ec8667d8ba0ae80227869e2
Parents: 61a3e7e
Author: Yiming Liu <li...@gmail.com>
Authored: Sun Sep 25 17:05:02 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 25 17:07:48 2016 +0800

----------------------------------------------------------------------
 .../test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b95e9671/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java b/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java
index dd52f82..c61f07f 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java
@@ -54,7 +54,7 @@ public class CubeSpecificConfigTest extends LocalFileMetadataTestCase {
     }
 
     private void verifyOverride(KylinConfig base, KylinConfig override) {
-        assertEquals("none", base.getHbaseDefaultCompressionCodec());
+        assertEquals("snappy", base.getHbaseDefaultCompressionCodec());
         assertEquals("lz4", override.getHbaseDefaultCompressionCodec());
     }
 }


[08/50] [abbrv] kylin git commit: Revert "KYLIN-1726 update to kafka 0.10"

Posted by li...@apache.org.
Revert "KYLIN-1726 update to kafka 0.10"

This reverts commit 1b1b2e37fdcba7ad67f0fa3f2369aa65431f13bc.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 355e58ba4209ddf945663228688d550bf654c387
Parents: da5ba27
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:50:26 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/cube/CubeManager.java | 30 ++++------------
 .../kylin/rest/controller/CubeController.java   |  8 ++---
 .../apache/kylin/rest/service/JobService.java   |  4 +--
 .../source/kafka/util/KafkaSampleProducer.java  | 38 ++++++++------------
 4 files changed, 27 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/355e58ba/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
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 11eabce..fc68798 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -679,28 +679,12 @@ public class CubeManager implements IRealizationProvider {
             return null;
         }
 
-        List<CubeSegment> buildingSegs = cube.getBuildingSegments();
-        if (buildingSegs.size() > 0) {
-            logger.debug("Cube " + cube.getName() + " has " + buildingSegs.size() + " building segments");
-        }
-
-        List<CubeSegment> readySegs = cube.getSegments(SegmentStatusEnum.READY);
-
-        List<CubeSegment> mergingSegs = Lists.newArrayList();
-        if (buildingSegs.size() > 0) {
-            
-            for (CubeSegment building : buildingSegs) {
-                // exclude those under-merging segs
-                for (CubeSegment ready : readySegs) {
-                    if (ready.getSourceOffsetStart() >= building.getSourceOffsetStart() && ready.getSourceOffsetEnd() <= building.getSourceOffsetEnd()) {
-                        mergingSegs.add(ready);
-                    }
-                }
-            }
+        if (cube.getBuildingSegments().size() > 0) {
+            logger.debug("Cube " + cube.getName() + " has bulding segment, will not trigger merge at this moment");
+            return null;
         }
 
-        // exclude those already under merging segments
-        readySegs.removeAll(mergingSegs);
+        List<CubeSegment> ready = cube.getSegments(SegmentStatusEnum.READY);
 
         long[] timeRanges = cube.getDescriptor().getAutoMergeTimeRanges();
         Arrays.sort(timeRanges);
@@ -708,9 +692,9 @@ public class CubeManager implements IRealizationProvider {
         for (int i = timeRanges.length - 1; i >= 0; i--) {
             long toMergeRange = timeRanges[i];
 
-            for (int s = 0; s < readySegs.size(); s++) {
-                CubeSegment seg = readySegs.get(s);
-                Pair<CubeSegment, CubeSegment> p = findMergeOffsetsByDateRange(readySegs.subList(s, readySegs.size()), //
+            for (int s = 0; s < ready.size(); s++) {
+                CubeSegment seg = ready.get(s);
+                Pair<CubeSegment, CubeSegment> p = findMergeOffsetsByDateRange(ready.subList(s, ready.size()), //
                         seg.getDateRangeStart(), seg.getDateRangeStart() + toMergeRange, toMergeRange);
                 if (p != null && p.getSecond().getDateRangeEnd() - p.getFirst().getDateRangeStart() >= toMergeRange)
                     return Pair.newPair(p.getFirst().getSourceOffsetStart(), p.getSecond().getSourceOffsetEnd());

http://git-wip-us.apache.org/repos/asf/kylin/blob/355e58ba/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
----------------------------------------------------------------------
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 669f53e..42b117c 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
@@ -272,7 +272,7 @@ public class CubeController extends BasicController {
     @RequestMapping(value = "/{cubeName}/rebuild", method = { RequestMethod.PUT })
     @ResponseBody
     public JobInstance rebuild(@PathVariable String cubeName, @RequestBody JobBuildRequest req) {
-        return buildInternal(cubeName, req.getStartTime(), req.getEndTime(), 0, 0, req.getBuildType(), true, req.isForce() || req.isForceMergeEmptySegment());
+        return buildInternal(cubeName, req.getStartTime(), req.getEndTime(), 0, 0, req.getBuildType(), req.isForce() || req.isForceMergeEmptySegment());
     }
 
     /** Build/Rebuild a cube segment by source offset */
@@ -286,16 +286,16 @@ public class CubeController extends BasicController {
     @RequestMapping(value = "/{cubeName}/rebuild2", method = { RequestMethod.PUT })
     @ResponseBody
     public JobInstance rebuild(@PathVariable String cubeName, @RequestBody JobBuildRequest2 req) {
-        return buildInternal(cubeName, 0, 0, req.getStartSourceOffset(), req.getEndSourceOffset(), req.getBuildType(), false, req.isForce());
+        return buildInternal(cubeName, 0, 0, req.getStartSourceOffset(), req.getEndSourceOffset(), req.getBuildType(), req.isForce());
     }
 
     private JobInstance buildInternal(String cubeName, long startTime, long endTime, //
-            long startOffset, long endOffset, String buildType, boolean strictCheck, boolean force) {
+            long startOffset, long endOffset, String buildType, boolean force) {
         try {
             String submitter = SecurityContextHolder.getContext().getAuthentication().getName();
             CubeInstance cube = jobService.getCubeManager().getCube(cubeName);
             return jobService.submitJob(cube, startTime, endTime, startOffset, endOffset, //
-                    CubeBuildTypeEnum.valueOf(buildType), strictCheck, force, submitter);
+                    CubeBuildTypeEnum.valueOf(buildType), force, submitter);
         } catch (Exception e) {
             logger.error(e.getLocalizedMessage(), e);
             throw new InternalErrorException(e.getLocalizedMessage());

http://git-wip-us.apache.org/repos/asf/kylin/blob/355e58ba/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
index 8929bf1..5c704ba 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -199,7 +199,7 @@ public class JobService extends BasicService {
 
     @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'OPERATION') or hasPermission(#cube, 'MANAGEMENT')")
     public JobInstance submitJob(CubeInstance cube, long startDate, long endDate, long startOffset, long endOffset, //
-            CubeBuildTypeEnum buildType, boolean strictCheck, boolean force, String submitter) throws IOException, JobException {
+            CubeBuildTypeEnum buildType, boolean force, String submitter) throws IOException, JobException {
 
         if (cube.getStatus() == RealizationStatusEnum.DESCBROKEN) {
             throw new BadRequestException("Broken cube " + cube.getName() + " can't be built");
@@ -211,7 +211,7 @@ public class JobService extends BasicService {
         DefaultChainedExecutable job;
 
         if (buildType == CubeBuildTypeEnum.BUILD) {
-            CubeSegment newSeg = getCubeManager().appendSegment(cube, startDate, endDate, startOffset, endOffset, strictCheck);
+            CubeSegment newSeg = getCubeManager().appendSegment(cube, startDate, endDate, startOffset, endOffset);
             job = EngineFactory.createBatchCubingJob(newSeg, submitter);
         } else if (buildType == CubeBuildTypeEnum.MERGE) {
             CubeSegment newSeg = getCubeManager().mergeSegments(cube, startDate, endDate, startOffset, endOffset, force);

http://git-wip-us.apache.org/repos/asf/kylin/blob/355e58ba/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaSampleProducer.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaSampleProducer.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaSampleProducer.java
index 3d26d3d..2a86a98 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaSampleProducer.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaSampleProducer.java
@@ -30,15 +30,16 @@ import java.util.Random;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 
+import kafka.javaapi.producer.Producer;
+import kafka.producer.KeyedMessage;
+import kafka.producer.ProducerConfig;
+
 /**
  * A sample producer which will create sample data to kafka topic
  */
@@ -48,8 +49,7 @@ public class KafkaSampleProducer {
     @SuppressWarnings("static-access")
     private static final Option OPTION_TOPIC = OptionBuilder.withArgName("topic").hasArg().isRequired(true).withDescription("Kafka topic").create("topic");
     private static final Option OPTION_BROKER = OptionBuilder.withArgName("broker").hasArg().isRequired(true).withDescription("Kafka broker").create("broker");
-    private static final Option OPTION_DELAY = OptionBuilder.withArgName("delay").hasArg().isRequired(false).withDescription("Simulated message delay in mili-seconds, default 0").create("delay");
-    private static final Option OPTION_INTERVAL = OptionBuilder.withArgName("interval").hasArg().isRequired(false).withDescription("Simulated message interval in mili-seconds, default 1000").create("interval");
+    private static final Option OPTION_DELAY = OptionBuilder.withArgName("delay").hasArg().isRequired(false).withDescription("Simulated message delay").create("delay");
 
     private static final ObjectMapper mapper = new ObjectMapper();
 
@@ -61,7 +61,6 @@ public class KafkaSampleProducer {
         options.addOption(OPTION_TOPIC);
         options.addOption(OPTION_BROKER);
         options.addOption(OPTION_DELAY);
-        options.addOption(OPTION_INTERVAL);
         optionsHelper.parseOptions(options, args);
 
         logger.info("options: '" + optionsHelper.getOptionsAsString() + "'");
@@ -71,13 +70,7 @@ public class KafkaSampleProducer {
         long delay = 0;
         String delayString = optionsHelper.getOptionValue(OPTION_DELAY);
         if (delayString != null) {
-            delay = Long.parseLong(delayString);
-        }
-
-        long interval = 1000;
-        String intervalString = optionsHelper.getOptionValue(OPTION_INTERVAL);
-        if (intervalString != null) {
-            interval = Long.parseLong(intervalString);
+            delay = Long.parseLong(optionsHelper.getOptionValue(OPTION_DELAY));
         }
 
         List<String> countries = new ArrayList();
@@ -102,16 +95,13 @@ public class KafkaSampleProducer {
         devices.add("Other");
 
         Properties props = new Properties();
-        props.put("bootstrap.servers", broker);
-        props.put("acks", "all");
-        props.put("retries", 0);
-        props.put("batch.size", 16384);
-        props.put("linger.ms", 1);
-        props.put("buffer.memory", 33554432);
-        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+        props.put("metadata.broker.list", broker);
+        props.put("serializer.class", "kafka.serializer.StringEncoder");
+        props.put("request.required.acks", "1");
+
+        ProducerConfig config = new ProducerConfig(props);
 
-        Producer<String, String> producer = new KafkaProducer<>(props);
+        Producer<String, String> producer = new Producer<String, String>(config);
 
         boolean alive = true;
         Random rnd = new Random();
@@ -124,10 +114,10 @@ public class KafkaSampleProducer {
             record.put("qty", rnd.nextInt(10));
             record.put("currency", "USD");
             record.put("amount", rnd.nextDouble() * 100);
-            ProducerRecord<String, String> data = new ProducerRecord<String, String>(topic, System.currentTimeMillis() + "", mapper.writeValueAsString(record));
+            KeyedMessage<String, String> data = new KeyedMessage<String, String>(topic, System.currentTimeMillis() + "", mapper.writeValueAsString(record));
             System.out.println("Sending 1 message");
             producer.send(data);
-            Thread.sleep(interval);
+            Thread.sleep(2000);
         }
         producer.close();
     }


[10/50] [abbrv] kylin git commit: KYLIN-2032 partition column shall not be always on dictionary

Posted by li...@apache.org.
KYLIN-2032 partition column shall not be always on dictionary


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 5a049a58de91d84dda67d7d422786c38f75f4e8e
Parents: a768b65
Author: Li Yang <li...@apache.org>
Authored: Tue Sep 20 18:40:28 2016 +0800
Committer: Li Yang <li...@apache.org>
Committed: Tue Sep 20 18:40:28 2016 +0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/kylin/cube/CubeManager.java     | 8 --------
 1 file changed, 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/5a049a58/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
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 daeca0d..fd46b54 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -937,14 +937,6 @@ public class CubeManager implements IRealizationProvider {
                 factDictCols.add(col);
             }
         }
-
-        // add partition column in all case
-        if (cubeDesc.getModel().getPartitionDesc() != null) {
-            TblColRef partitionCol = cubeDesc.getModel().getPartitionDesc().getPartitionDateColumnRef();
-            if (factDictCols.contains(partitionCol) == false) {
-                factDictCols.add(partitionCol);
-            }
-        }
         return factDictCols;
     }
 }


[04/50] [abbrv] kylin git commit: Revert "KYLIN-1762 discard job when no stream message"

Posted by li...@apache.org.
Revert "KYLIN-1762 discard job when no stream message"

This reverts commit 1108d9eeccecbccffea0b3f9049151672196c91a.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: da5ba276b972f8b3c0d220252e74ac2ff73298fc
Parents: bec25b4
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:50:20 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../job/execution/DefaultChainedExecutable.java |  6 ---
 .../kylin/source/kafka/SeekOffsetStep.java      | 45 +++++---------------
 2 files changed, 10 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/da5ba276/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
index 39a5f4f..753b389 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
@@ -88,7 +88,6 @@ public class DefaultChainedExecutable extends AbstractExecutable implements Chai
             boolean allSucceed = true;
             boolean hasError = false;
             boolean hasRunning = false;
-            boolean hasDiscarded = false;
             for (Executable task : jobs) {
                 final ExecutableState status = task.getStatus();
                 if (status == ExecutableState.ERROR) {
@@ -100,9 +99,6 @@ public class DefaultChainedExecutable extends AbstractExecutable implements Chai
                 if (status == ExecutableState.RUNNING) {
                     hasRunning = true;
                 }
-                if (status == ExecutableState.DISCARDED) {
-                    hasDiscarded = true;
-                }
             }
             if (allSucceed) {
                 setEndTime(System.currentTimeMillis());
@@ -114,8 +110,6 @@ public class DefaultChainedExecutable extends AbstractExecutable implements Chai
                 notifyUserStatusChange(executableContext, ExecutableState.ERROR);
             } else if (hasRunning) {
                 jobService.updateJobOutput(getId(), ExecutableState.RUNNING, null, null);
-            } else if (hasDiscarded) {
-                jobService.updateJobOutput(getId(), ExecutableState.DISCARDED, null, null);
             } else {
                 jobService.updateJobOutput(getId(), ExecutableState.READY, null, null);
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/da5ba276/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
index 479f1b8..5dca93f 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
@@ -17,10 +17,6 @@
 */
 package org.apache.kylin.source.kafka;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.Maps;
-import org.apache.commons.math3.util.MathUtils;
 import org.apache.kylin.source.kafka.util.KafkaClient;
 import org.apache.kylin.source.kafka.util.KafkaOffsetMapping;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
@@ -38,7 +34,6 @@ import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -106,39 +101,19 @@ public class SeekOffsetStep extends AbstractExecutable {
             }
         }
 
-        long totalStartOffset = 0, totalEndOffset = 0;
-        for (Long v : startOffsets.values()) {
-            totalStartOffset += v;
-        }
-        for (Long v : endOffsets.values()) {
-            totalEndOffset += v;
-        }
+        KafkaOffsetMapping.saveOffsetStart(segment, startOffsets);
+        KafkaOffsetMapping.saveOffsetEnd(segment, endOffsets);
 
-        if (totalEndOffset > totalStartOffset) {
-            KafkaOffsetMapping.saveOffsetStart(segment, startOffsets);
-            KafkaOffsetMapping.saveOffsetEnd(segment, endOffsets);
-            segment.setName(CubeSegment.makeSegmentName(0, 0, totalStartOffset, totalEndOffset));
-            CubeUpdate cubeBuilder = new CubeUpdate(cube);
-            cubeBuilder.setToUpdateSegs(segment);
-            try {
-                cubeManager.updateCube(cubeBuilder);
-            } catch (IOException e) {
-                return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-            }
+        segment.setName(CubeSegment.makeSegmentName(0, 0, segment.getSourceOffsetStart(), segment.getSourceOffsetEnd()));
+        CubeUpdate cubeBuilder = new CubeUpdate(cube);
+        cubeBuilder.setToUpdateSegs(segment);
+        try {
+            cubeManager.updateCube(cubeBuilder);
             return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-        } else {
-            CubeUpdate cubeBuilder = new CubeUpdate(cube);
-            cubeBuilder.setToRemoveSegs(segment);
-            try {
-                cubeManager.updateCube(cubeBuilder);
-            } catch (IOException e) {
-                return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
-            }
-
-            return new ExecuteResult(ExecuteResult.State.DISCARDED, "No new message comes");
+        } catch (IOException e) {
+            logger.error("fail to update cube segment offset", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
         }
-
-
     }
 
 }


[37/50] [abbrv] kylin git commit: KYLIN-2033 Broadcaster stronger sync locking and more comments

Posted by li...@apache.org.
KYLIN-2033 Broadcaster stronger sync locking and more comments


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: e10f2b922006a002ea9cb58ff11a4ecd9aa749c9
Parents: bf127a9
Author: Yang Li <li...@apache.org>
Authored: Sun Sep 25 13:54:51 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Sep 25 13:54:51 2016 +0800

----------------------------------------------------------------------
 .../kylin/metadata/cachesync/Broadcaster.java   | 119 ++++++++++---------
 1 file changed, 65 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e10f2b92/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
index 75b2333..8d34cc0 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
@@ -43,7 +43,16 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 /**
- * Broadcast kylin event out
+ * Broadcast metadata changes across all Kylin servers.
+ * 
+ * The origin server announce the event via Rest API to all Kylin servers including itself.
+ * On target server, listeners are registered to process events. As part of processing, a 
+ * listener can re-notify a new event to other local listeners.
+ * 
+ * A typical project schema change event:
+ * - model is update on origin server, a "model" update event is announced
+ * - on all servers, model listener is invoked, reload the model, and notify a "project_schema" update event
+ * - all listeners respond to the "project_schema" update -- reload cube desc, clear project L2 cache, clear calcite data source etc
  */
 public class Broadcaster {
 
@@ -57,13 +66,9 @@ public class Broadcaster {
     private static final ConcurrentHashMap<KylinConfig, Broadcaster> CACHE = new ConcurrentHashMap<KylinConfig, Broadcaster>();
 
     public static Broadcaster getInstance(KylinConfig config) {
-        Broadcaster r = CACHE.get(config);
-        if (r != null) {
-            return r;
-        }
 
-        synchronized (Broadcaster.class) {
-            r = CACHE.get(config);
+        synchronized (CACHE) {
+            Broadcaster r = CACHE.get(config);
             if (r != null) {
                 return r;
             }
@@ -79,7 +84,9 @@ public class Broadcaster {
 
     // call Broadcaster.getInstance().notifyClearAll() to clear cache
     static void clearCache() {
-        CACHE.clear();
+        synchronized (CACHE) {
+            CACHE.clear();
+        }
     }
 
     // ============================================================================
@@ -134,22 +141,24 @@ public class Broadcaster {
     }
 
     public void registerListener(Listener listener, String... entities) {
-        // ignore re-registration
-        List<Listener> all = listenerMap.get(SYNC_ALL);
-        if (all != null && all.contains(listener)) {
-            return;
-        }
+        synchronized (CACHE) {
+            // ignore re-registration
+            List<Listener> all = listenerMap.get(SYNC_ALL);
+            if (all != null && all.contains(listener)) {
+                return;
+            }
 
-        for (String entity : entities) {
-            if (!StringUtils.isBlank(entity))
-                addListener(entity, listener);
+            for (String entity : entities) {
+                if (!StringUtils.isBlank(entity))
+                    addListener(entity, listener);
+            }
+            addListener(SYNC_ALL, listener);
+            addListener(SYNC_PRJ_SCHEMA, listener);
+            addListener(SYNC_PRJ_DATA, listener);
         }
-        addListener(SYNC_ALL, listener);
-        addListener(SYNC_PRJ_SCHEMA, listener);
-        addListener(SYNC_PRJ_DATA, listener);
     }
 
-    synchronized private void addListener(String entity, Listener listener) {
+    private void addListener(String entity, Listener listener) {
         List<Listener> list = listenerMap.get(entity);
         if (list == null) {
             list = new ArrayList<>();
@@ -170,42 +179,44 @@ public class Broadcaster {
         notifyListener(SYNC_PRJ_DATA, Event.UPDATE, project);
     }
 
-    public synchronized void notifyListener(String entity, Event event, String cacheKey) throws IOException {
-        List<Listener> list = listenerMap.get(entity);
-        if (list == null)
-            return;
-        
-        logger.debug("Broadcasting metadata change: entity=" + entity + ", event=" + event + ", cacheKey=" + cacheKey + ", listeners=" + list);
-        
-        // prevents concurrent modification exception
-        list = Lists.newArrayList(list);
-        switch (entity) {
-        case SYNC_ALL:
-            for (Listener l : list) {
-                l.onClearAll(this);
-            }
-            clearCache(); // clear broadcaster too in the end
-            break;
-        case SYNC_PRJ_SCHEMA:
-            ProjectManager.getInstance(config).clearL2Cache();
-            for (Listener l : list) {
-                l.onProjectSchemaChange(this, cacheKey);
-            }
-            break;
-        case SYNC_PRJ_DATA:
-            ProjectManager.getInstance(config).clearL2Cache(); // cube's first becoming ready leads to schema change too
-            for (Listener l : list) {
-                l.onProjectDataChange(this, cacheKey);
-            }
-            break;
-        default:
-            for (Listener l : list) {
-                l.onEntityChange(this, entity, event, cacheKey);
+    public void notifyListener(String entity, Event event, String cacheKey) throws IOException {
+        synchronized (CACHE) {
+            List<Listener> list = listenerMap.get(entity);
+            if (list == null)
+                return;
+
+            logger.debug("Broadcasting metadata change: entity=" + entity + ", event=" + event + ", cacheKey=" + cacheKey + ", listeners=" + list);
+
+            // prevents concurrent modification exception
+            list = Lists.newArrayList(list);
+            switch (entity) {
+            case SYNC_ALL:
+                for (Listener l : list) {
+                    l.onClearAll(this);
+                }
+                clearCache(); // clear broadcaster too in the end
+                break;
+            case SYNC_PRJ_SCHEMA:
+                ProjectManager.getInstance(config).clearL2Cache();
+                for (Listener l : list) {
+                    l.onProjectSchemaChange(this, cacheKey);
+                }
+                break;
+            case SYNC_PRJ_DATA:
+                ProjectManager.getInstance(config).clearL2Cache(); // cube's first becoming ready leads to schema change too
+                for (Listener l : list) {
+                    l.onProjectDataChange(this, cacheKey);
+                }
+                break;
+            default:
+                for (Listener l : list) {
+                    l.onEntityChange(this, entity, event, cacheKey);
+                }
+                break;
             }
-            break;
+
+            logger.debug("Done broadcasting metadata change: entity=" + entity + ", event=" + event + ", cacheKey=" + cacheKey);
         }
-        
-        logger.debug("Done broadcasting metadata change: entity=" + entity + ", event=" + event + ", cacheKey=" + cacheKey);
     }
 
     /**


[33/50] [abbrv] kylin git commit: minor, more logging in storage query

Posted by li...@apache.org.
minor, more logging in storage query


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 5658283de1dd653881eacd8442ae6f3122600581
Parents: 5aeedd5
Author: lidongsjtu <li...@apache.org>
Authored: Sat Sep 24 10:26:44 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Sat Sep 24 10:42:46 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/5658283d/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/GTCubeStorageQueryBase.java
----------------------------------------------------------------------
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 31663d0..4ba134d 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
@@ -103,6 +103,7 @@ public abstract class GTCubeStorageQueryBase implements IStorageQuery {
         dimensionsD.addAll(groupsD);
         dimensionsD.addAll(otherDimsD);
         Cuboid cuboid = Cuboid.identifyCuboid(cubeDesc, dimensionsD, metrics);
+        logger.info("Cuboid identified: cube={}, cuboidId={}", cubeInstance.getName(), cuboid.getId());
         context.setCuboid(cuboid);
 
         // isExactAggregation? meaning: tuples returned from storage requires no further aggregation in query engine


[42/50] [abbrv] kylin git commit: KYLIN-1565: Read KV max size from HBase config

Posted by li...@apache.org.
KYLIN-1565: Read KV max size from HBase config


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: e05cb60061a43987f7648d3d04921b5a01bb5ac9
Parents: 0861bfb
Author: Yiming Liu <li...@gmail.com>
Authored: Fri Sep 16 11:51:11 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Mon Sep 26 19:45:19 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/common/KylinConfigBase.java     |  4 ----
 .../kylin/storage/hbase/ITHBaseResourceStoreTest.java     | 10 +++++-----
 .../apache/kylin/storage/hbase/HBaseResourceStore.java    |  2 +-
 3 files changed, 6 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/e05cb600/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 79ee084..ceb188e 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -570,10 +570,6 @@ abstract public class KylinConfigBase implements Serializable {
         return getOptionalIntArray("kylin.query.metrics.percentiles.intervals", dft);
     }
 
-    public int getHBaseKeyValueSize() {
-        return Integer.parseInt(this.getOptional("kylin.hbase.client.keyvalue.maxsize", "10485760"));
-    }
-
     public String getDefaultIGTStorage() {
         return getOptional("kylin.query.storage.default.gtstorage", "org.apache.kylin.storage.hbase.cube.v2.CubeHBaseEndpointRPC");
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e05cb600/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
index bc5cdf1..5a2d5dd 100644
--- a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
@@ -57,12 +57,12 @@ public class ITHBaseResourceStoreTest extends HBaseMetadataTestCase {
         String path = "/cube/_test_large_cell.json";
         String largeContent = "THIS_IS_A_LARGE_CELL";
         StringEntity content = new StringEntity(largeContent);
-        KylinConfig config = KylinConfig.getInstanceFromEnv();
-        int origSize = config.getHBaseKeyValueSize();
+        Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
+        int origSize = Integer.parseInt(hconf.get("hbase.client.keyvalue.maxsize", "10485760"));
         ResourceStore store = ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
 
         try {
-            config.setProperty("kylin.hbase.client.keyvalue.maxsize", String.valueOf(largeContent.length() - 1));
+            hconf.set("hbase.client.keyvalue.maxsize", String.valueOf(largeContent.length() - 1));
 
             store.deleteResource(path);
 
@@ -72,7 +72,7 @@ public class ITHBaseResourceStoreTest extends HBaseMetadataTestCase {
             assertEquals(content, t);
 
             Path redirectPath = ((HBaseResourceStore) store).bigCellHDFSPath(path);
-            Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
+
             FileSystem fileSystem = FileSystem.get(hconf);
             assertTrue(fileSystem.exists(redirectPath));
 
@@ -82,7 +82,7 @@ public class ITHBaseResourceStoreTest extends HBaseMetadataTestCase {
 
             store.deleteResource(path);
         } finally {
-            config.setProperty("kylin.hbase.client.keyvalue.maxsize", "" + origSize);
+            hconf.set("hbase.client.keyvalue.maxsize", "" + origSize);
             store.deleteResource(path);
         }
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e05cb600/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
index e2f3661..ee4b7e4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
@@ -364,7 +364,7 @@ public class HBaseResourceStore extends ResourceStore {
     }
 
     private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
-        int kvSizeLimit = this.kylinConfig.getHBaseKeyValueSize();
+        int kvSizeLimit = Integer.parseInt(getConnection().getConfiguration().get("hbase.client.keyvalue.maxsize", "10485760"));
         if (content.length > kvSizeLimit) {
             writeLargeCellToHdfs(resPath, content, table);
             content = BytesUtil.EMPTY_BYTE_ARRAY;


[43/50] [abbrv] kylin git commit: KYLIN-1995: Upgrade deprecated properties for Hadoop 2.6

Posted by li...@apache.org.
KYLIN-1995: Upgrade deprecated properties for Hadoop 2.6


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 2993e7bdd8875e6b2f8893d7852bb67d226f6c9e
Parents: e05cb60
Author: Yiming Liu <li...@gmail.com>
Authored: Sun Sep 4 17:31:45 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Mon Sep 26 19:51:04 2016 +0800

----------------------------------------------------------------------
 build/conf/kylin_hive_conf.xml      |  2 +-
 build/conf/kylin_job_conf.xml       | 12 ++++++------
 build/conf/kylin_job_conf_inmem.xml | 12 ++++++------
 3 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/2993e7bd/build/conf/kylin_hive_conf.xml
----------------------------------------------------------------------
diff --git a/build/conf/kylin_hive_conf.xml b/build/conf/kylin_hive_conf.xml
index 30c4feb..4a1b3b6 100644
--- a/build/conf/kylin_hive_conf.xml
+++ b/build/conf/kylin_hive_conf.xml
@@ -62,7 +62,7 @@
     </property>
     -->
     <property>
-        <name>mapred.output.compression.type</name>
+        <name>mapreduce.output.fileoutputformat.compress.type</name>
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2993e7bd/build/conf/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/build/conf/kylin_job_conf.xml b/build/conf/kylin_job_conf.xml
index 96b806c..17a9145 100644
--- a/build/conf/kylin_job_conf.xml
+++ b/build/conf/kylin_job_conf.xml
@@ -26,7 +26,7 @@
     </property>
 
     <property>
-        <name>mapred.compress.map.output</name>
+        <name>mapreduce.map.output.compress</name>
         <value>true</value>
         <description>Compress map outputs</description>
     </property>
@@ -37,14 +37,14 @@
     -->
     <!--
     <property>
-        <name>mapred.map.output.compression.codec</name>
+        <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for map outputs
         </description>
     </property>
     -->
     <property>
-        <name>mapred.output.compress</name>
+        <name>mapreduce.output.fileoutputformat.compress</name>
         <value>true</value>
         <description>Compress the output of a MapReduce job</description>
     </property>
@@ -54,14 +54,14 @@
     -->
     <!--
     <property>
-        <name>mapred.output.compression.codec</name>
+        <name>mapreduce.output.fileoutputformat.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for job outputs
         </description>
     </property>
     -->
     <property>
-        <name>mapred.output.compression.type</name>
+        <name>mapreduce.output.fileoutputformat.compress.type</name>
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>
@@ -80,7 +80,7 @@
     </property>
 
     <property>
-        <name>mapred.task.timeout</name>
+        <name>mapreduce.task.timeout</name>
         <value>3600000</value>
         <description>Set task timeout to 1 hour</description>
     </property>

http://git-wip-us.apache.org/repos/asf/kylin/blob/2993e7bd/build/conf/kylin_job_conf_inmem.xml
----------------------------------------------------------------------
diff --git a/build/conf/kylin_job_conf_inmem.xml b/build/conf/kylin_job_conf_inmem.xml
index fea2f68..1d349f6 100644
--- a/build/conf/kylin_job_conf_inmem.xml
+++ b/build/conf/kylin_job_conf_inmem.xml
@@ -26,7 +26,7 @@
     </property>
 
     <property>
-        <name>mapred.compress.map.output</name>
+        <name>mapreduce.map.output.compress</name>
         <value>true</value>
         <description>Compress map outputs</description>
     </property>
@@ -37,14 +37,14 @@
     -->
     <!--
     <property>
-        <name>mapred.map.output.compression.codec</name>
+        <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for map outputs
         </description>
     </property>
     -->
     <property>
-        <name>mapred.output.compress</name>
+        <name>mapreduce.output.fileoutputformat.compress</name>
         <value>true</value>
         <description>Compress the output of a MapReduce job</description>
     </property>
@@ -54,14 +54,14 @@
     -->
     <!--
     <property>
-        <name>mapred.output.compression.codec</name>
+        <name>mapreduce.output.fileoutputformat.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for job outputs
         </description>
     </property>
     -->
     <property>
-        <name>mapred.output.compression.type</name>
+        <name>mapreduce.output.fileoutputformat.compress.type</name>
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>
@@ -80,7 +80,7 @@
     </property>
 
     <property>
-        <name>mapred.task.timeout</name>
+        <name>mapreduce.task.timeout</name>
         <value>3600000</value>
         <description>Set task timeout to 1 hour</description>
     </property>


[09/50] [abbrv] kylin git commit: KYLIN-2010 retire 'time' and 'date' encoding, rollback to use dictionary

Posted by li...@apache.org.
KYLIN-2010 retire 'time' and 'date' encoding, rollback to use dictionary


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: a768b65d608cb6c1d0dbb7bcadf4adbd97a7b1f6
Parents: 506cd78
Author: Li Yang <li...@apache.org>
Authored: Tue Sep 20 18:29:20 2016 +0800
Committer: Li Yang <li...@apache.org>
Committed: Tue Sep 20 18:30:14 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/cube/model/RowKeyColDesc.java  | 15 ++++--------
 .../org/apache/kylin/dimension/DateDimEnc.java  | 24 +++++++++++---------
 .../dimension/DimensionEncodingFactory.java     |  2 --
 .../org/apache/kylin/dimension/TimeDimEnc.java  | 24 +++++++++++---------
 4 files changed, 30 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a768b65d/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
index 8d4f862..296c1a3 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
@@ -26,7 +26,6 @@ import org.apache.kylin.dimension.DictionaryDimEnc;
 import org.apache.kylin.dimension.DimensionEncoding;
 import org.apache.kylin.dimension.DimensionEncodingFactory;
 import org.apache.kylin.dimension.TimeDimEnc;
-import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -38,8 +37,8 @@ import com.google.common.base.Preconditions;
 
 /**
  * @author yangli9
- * 
  */
+@SuppressWarnings("deprecation")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class RowKeyColDesc {
 
@@ -75,15 +74,9 @@ public class RowKeyColDesc {
         if (!DimensionEncodingFactory.isVaildEncoding(this.encodingName))
             throw new IllegalArgumentException("Not supported row key col encoding: '" + this.encoding + "'");
 
-        // convert date/time dictionary to DimensionEncoding implicitly, date/time dictionary is deprecated
-        if (DictionaryDimEnc.ENCODING_NAME.equals(encodingName)) {
-            DataType type = colRef.getType();
-            if (type.isDate()) {
-                encoding = encodingName = DateDimEnc.ENCODING_NAME;
-            }
-            if (type.isTime() || type.isTimestamp() || type.isDatetime()) {
-                encoding = encodingName = TimeDimEnc.ENCODING_NAME;
-            }
+        // convert date/time encoding to dictionary, because encoding does not remember date pattern and cannot format string correctly at query time
+        if (DateDimEnc.ENCODING_NAME.equals(encodingName) || TimeDimEnc.ENCODING_NAME.equals(encodingName)) {
+            throw new IllegalStateException("Encoding '" + DateDimEnc.ENCODING_NAME + "' and '" + TimeDimEnc.ENCODING_NAME + "' are deprecated, please use dictionary encoding instead");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/a768b65d/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
index 79e00ce..3992ed7 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
@@ -20,6 +20,8 @@ package org.apache.kylin.dimension;
 
 /**
  * This encoding is meant to be IDENTICAL to DateStrDictionary for 100% backward compatibility.
+ * 
+ * @deprecated for now, please use dictionary 
  */
 public class DateDimEnc extends AbstractDateDimEnc {
     private static final long serialVersionUID = 1L;
@@ -28,17 +30,17 @@ public class DateDimEnc extends AbstractDateDimEnc {
 
     public static final String ENCODING_NAME = "date";
 
-    public static class Factory extends DimensionEncodingFactory {
-        @Override
-        public String getSupportedEncodingName() {
-            return ENCODING_NAME;
-        }
-
-        @Override
-        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
-            return new DateDimEnc();
-        }
-    };
+//    public static class Factory extends DimensionEncodingFactory {
+//        @Override
+//        public String getSupportedEncodingName() {
+//            return ENCODING_NAME;
+//        }
+//
+//        @Override
+//        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+//            return new DateDimEnc();
+//        }
+//    };
 
     public DateDimEnc() {
         super(3, new IMillisCodec() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/a768b65d/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java b/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
index 739fa25..bc4bf5c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
@@ -77,8 +77,6 @@ public abstract class DimensionEncodingFactory {
             map.put(IntegerDimEnc.ENCODING_NAME, new IntegerDimEnc.Factory());
             map.put(SlimLongDimEnc.ENCODING_NAME, new SlimLongDimEnc.Factory());
             map.put(FixedLenHexDimEnc.ENCODING_NAME, new FixedLenHexDimEnc.Factory());
-            map.put(DateDimEnc.ENCODING_NAME, new DateDimEnc.Factory());
-            map.put(TimeDimEnc.ENCODING_NAME, new TimeDimEnc.Factory());
 
             // custom encodings
             String[] clsNames = KylinConfig.getInstanceFromEnv().getCubeDimensionCustomEncodingFactories();

http://git-wip-us.apache.org/repos/asf/kylin/blob/a768b65d/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
index 8c66d5d..245f513 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
@@ -20,23 +20,25 @@ package org.apache.kylin.dimension;
 
 /**
  * This encoding is meant to be IDENTICAL to TimeStrDictionary for 100% backward compatibility.
+ * 
+ * @deprecated for now, please use dictionary 
  */
 public class TimeDimEnc extends AbstractDateDimEnc {
     private static final long serialVersionUID = 1L;
 
     public static final String ENCODING_NAME = "time";
 
-    public static class Factory extends DimensionEncodingFactory {
-        @Override
-        public String getSupportedEncodingName() {
-            return ENCODING_NAME;
-        }
-
-        @Override
-        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
-            return new TimeDimEnc();
-        }
-    };
+//    public static class Factory extends DimensionEncodingFactory {
+//        @Override
+//        public String getSupportedEncodingName() {
+//            return ENCODING_NAME;
+//        }
+//
+//        @Override
+//        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+//            return new TimeDimEnc();
+//        }
+//    };
 
     public TimeDimEnc() {
         super(4, new IMillisCodec() {


[15/50] [abbrv] kylin git commit: KYLIN-2010 backend check date/time encoding only apply to date/time column

Posted by li...@apache.org.
KYLIN-2010 backend check date/time encoding only apply to date/time column


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: f0d2be3c93cde5e658178c896be1671d1aca5685
Parents: f62541e
Author: Yang Li <li...@apache.org>
Authored: Wed Sep 21 06:41:51 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Wed Sep 21 06:41:51 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/cube/model/RowKeyColDesc.java     | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/f0d2be3c/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
index 8d4f862..a1d4440 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
@@ -75,9 +75,10 @@ public class RowKeyColDesc {
         if (!DimensionEncodingFactory.isVaildEncoding(this.encodingName))
             throw new IllegalArgumentException("Not supported row key col encoding: '" + this.encoding + "'");
 
-        // convert date/time dictionary to DimensionEncoding implicitly, date/time dictionary is deprecated
+        // convert date/time dictionary on date/time column to DimensionEncoding implicitly
+        // however date/time dictionary on varchar column is still required
+        DataType type = colRef.getType();
         if (DictionaryDimEnc.ENCODING_NAME.equals(encodingName)) {
-            DataType type = colRef.getType();
             if (type.isDate()) {
                 encoding = encodingName = DateDimEnc.ENCODING_NAME;
             }
@@ -85,6 +86,10 @@ public class RowKeyColDesc {
                 encoding = encodingName = TimeDimEnc.ENCODING_NAME;
             }
         }
+        if (DateDimEnc.ENCODING_NAME.equals(encodingName) && type.isDate() == false)
+            throw new IllegalArgumentException(colRef + " type is " + type + " and cannot apply date encoding");
+        if (TimeDimEnc.ENCODING_NAME.equals(encodingName) && type.isTime() == false)
+            throw new IllegalArgumentException(colRef + " type is " + type + " and cannot apply time encoding");
     }
 
     public String getEncoding() {


[11/50] [abbrv] kylin git commit: Revert "KYLIN-2010 retire 'time' and 'date' encoding, rollback to use dictionary"

Posted by li...@apache.org.
Revert "KYLIN-2010 retire 'time' and 'date' encoding, rollback to use dictionary"

This reverts commit a768b65d608cb6c1d0dbb7bcadf4adbd97a7b1f6.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 7d716dff10326e8f906bdcb8d70c67fde18881d5
Parents: 5a049a5
Author: Yang Li <li...@apache.org>
Authored: Tue Sep 20 21:07:39 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Tue Sep 20 21:07:39 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/cube/model/RowKeyColDesc.java  | 15 ++++++++----
 .../org/apache/kylin/dimension/DateDimEnc.java  | 24 +++++++++-----------
 .../dimension/DimensionEncodingFactory.java     |  2 ++
 .../org/apache/kylin/dimension/TimeDimEnc.java  | 24 +++++++++-----------
 4 files changed, 35 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/7d716dff/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
index 296c1a3..8d4f862 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
@@ -26,6 +26,7 @@ import org.apache.kylin.dimension.DictionaryDimEnc;
 import org.apache.kylin.dimension.DimensionEncoding;
 import org.apache.kylin.dimension.DimensionEncodingFactory;
 import org.apache.kylin.dimension.TimeDimEnc;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.TblColRef;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -37,8 +38,8 @@ import com.google.common.base.Preconditions;
 
 /**
  * @author yangli9
+ * 
  */
-@SuppressWarnings("deprecation")
 @JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
 public class RowKeyColDesc {
 
@@ -74,9 +75,15 @@ public class RowKeyColDesc {
         if (!DimensionEncodingFactory.isVaildEncoding(this.encodingName))
             throw new IllegalArgumentException("Not supported row key col encoding: '" + this.encoding + "'");
 
-        // convert date/time encoding to dictionary, because encoding does not remember date pattern and cannot format string correctly at query time
-        if (DateDimEnc.ENCODING_NAME.equals(encodingName) || TimeDimEnc.ENCODING_NAME.equals(encodingName)) {
-            throw new IllegalStateException("Encoding '" + DateDimEnc.ENCODING_NAME + "' and '" + TimeDimEnc.ENCODING_NAME + "' are deprecated, please use dictionary encoding instead");
+        // convert date/time dictionary to DimensionEncoding implicitly, date/time dictionary is deprecated
+        if (DictionaryDimEnc.ENCODING_NAME.equals(encodingName)) {
+            DataType type = colRef.getType();
+            if (type.isDate()) {
+                encoding = encodingName = DateDimEnc.ENCODING_NAME;
+            }
+            if (type.isTime() || type.isTimestamp() || type.isDatetime()) {
+                encoding = encodingName = TimeDimEnc.ENCODING_NAME;
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d716dff/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
index 3992ed7..79e00ce 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/DateDimEnc.java
@@ -20,8 +20,6 @@ package org.apache.kylin.dimension;
 
 /**
  * This encoding is meant to be IDENTICAL to DateStrDictionary for 100% backward compatibility.
- * 
- * @deprecated for now, please use dictionary 
  */
 public class DateDimEnc extends AbstractDateDimEnc {
     private static final long serialVersionUID = 1L;
@@ -30,17 +28,17 @@ public class DateDimEnc extends AbstractDateDimEnc {
 
     public static final String ENCODING_NAME = "date";
 
-//    public static class Factory extends DimensionEncodingFactory {
-//        @Override
-//        public String getSupportedEncodingName() {
-//            return ENCODING_NAME;
-//        }
-//
-//        @Override
-//        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
-//            return new DateDimEnc();
-//        }
-//    };
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new DateDimEnc();
+        }
+    };
 
     public DateDimEnc() {
         super(3, new IMillisCodec() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d716dff/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java b/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
index bc4bf5c..739fa25 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
@@ -77,6 +77,8 @@ public abstract class DimensionEncodingFactory {
             map.put(IntegerDimEnc.ENCODING_NAME, new IntegerDimEnc.Factory());
             map.put(SlimLongDimEnc.ENCODING_NAME, new SlimLongDimEnc.Factory());
             map.put(FixedLenHexDimEnc.ENCODING_NAME, new FixedLenHexDimEnc.Factory());
+            map.put(DateDimEnc.ENCODING_NAME, new DateDimEnc.Factory());
+            map.put(TimeDimEnc.ENCODING_NAME, new TimeDimEnc.Factory());
 
             // custom encodings
             String[] clsNames = KylinConfig.getInstanceFromEnv().getCubeDimensionCustomEncodingFactories();

http://git-wip-us.apache.org/repos/asf/kylin/blob/7d716dff/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
index 245f513..8c66d5d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/TimeDimEnc.java
@@ -20,25 +20,23 @@ package org.apache.kylin.dimension;
 
 /**
  * This encoding is meant to be IDENTICAL to TimeStrDictionary for 100% backward compatibility.
- * 
- * @deprecated for now, please use dictionary 
  */
 public class TimeDimEnc extends AbstractDateDimEnc {
     private static final long serialVersionUID = 1L;
 
     public static final String ENCODING_NAME = "time";
 
-//    public static class Factory extends DimensionEncodingFactory {
-//        @Override
-//        public String getSupportedEncodingName() {
-//            return ENCODING_NAME;
-//        }
-//
-//        @Override
-//        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
-//            return new TimeDimEnc();
-//        }
-//    };
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new TimeDimEnc();
+        }
+    };
 
     public TimeDimEnc() {
         super(4, new IMillisCodec() {


[41/50] [abbrv] kylin git commit: For integration test, don’t use snappy compress for outputformat

Posted by li...@apache.org.
For integration test, don\u2019t use snappy compress for outputformat

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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 0861bfb0312d344267eaf5772ed1b3011f778301
Parents: b95e967
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 26 11:55:31 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Sep 26 11:55:31 2016 +0800

----------------------------------------------------------------------
 examples/test_case_data/sandbox/kylin_hive_conf.xml | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0861bfb0/examples/test_case_data/sandbox/kylin_hive_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_hive_conf.xml b/examples/test_case_data/sandbox/kylin_hive_conf.xml
index 4ee5f6b..593d4f8 100644
--- a/examples/test_case_data/sandbox/kylin_hive_conf.xml
+++ b/examples/test_case_data/sandbox/kylin_hive_conf.xml
@@ -45,12 +45,13 @@
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description></description>
     </property>
+    <!--
     <property>
         <name>mapreduce.output.fileoutputformat.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description></description>
     </property>
-
+    -->
     <property>
         <name>hive.merge.size.per.task</name>
         <value>32000000</value>


[47/50] [abbrv] kylin git commit: KYLIN-2019: Enable Apache Licence checker for Checkstyle

Posted by li...@apache.org.
KYLIN-2019: Enable Apache Licence checker for Checkstyle

Signed-off-by: Yang Li <li...@apache.org>


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 8596af11e9f46b7be82009f446b8adcca0d4e499
Parents: abc6a74
Author: Yiming Liu <li...@gmail.com>
Authored: Mon Sep 26 20:17:42 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Mon Sep 26 21:45:37 2016 +0800

----------------------------------------------------------------------
 .../main/config/assemblies/source-assembly.xml  | 117 ++++++
 checkstyle-suppressions.xml                     |  28 --
 checkstyle.xml                                  |  62 ---
 .../org/apache/kylin/common/KylinVersion.java   |  24 +-
 .../org/apache/kylin/common/util/Bytes.java     |   5 +-
 .../common/util/MemoryBudgetController.java     |  27 +-
 .../java/org/apache/kylin/common/util/Pair.java |   5 +-
 .../kylin/common/util/StreamingBatch.java       |  49 +--
 .../cube/gridtable/TrimmedCubeCodeSystem.java   |  32 +-
 .../inmemcubing/AbstractInMemCubeBuilder.java   |  28 +-
 .../cube/inmemcubing/CompoundCuboidWriter.java  |  27 +-
 .../cube/inmemcubing/ConcurrentDiskStore.java   |  27 +-
 .../cube/inmemcubing/DoggedCubeBuilder.java     |  28 +-
 .../kylin/cube/inmemcubing/ICuboidWriter.java   |  27 +-
 .../cube/inmemcubing/InMemCubeBuilder.java      |  28 +-
 .../kylin/cube/inmemcubing/MemDiskStore.java    |  27 +-
 .../org/apache/kylin/cube/util/CubingUtils.java |  49 +--
 .../org/apache/kylin/gridtable/IGTStorage.java  |  32 +-
 .../apache/kylin/gridtable/ScannerWorker.java   |  32 +-
 .../apache/kylin/gridtable/UnitTestSupport.java |  27 +-
 dev-support/checkstyle-java-header.txt          |  17 +
 dev-support/checkstyle-suppressions.xml         |  28 ++
 dev-support/checkstyle.xml                      |  67 ++++
 .../kylin/engine/streaming/IStreamingInput.java |  49 +--
 .../engine/streaming/IStreamingOutput.java      |  49 +--
 .../streaming/OneOffStreamingBuilder.java       |  49 +--
 .../engine/streaming/StreamingBatchBuilder.java |  49 +--
 .../kylin/engine/streaming/StreamingConfig.java |  48 +--
 .../engine/streaming/StreamingManager.java      |  48 +--
 .../engine/streaming/cli/StreamingCLI.java      |  48 +--
 .../streaming/cube/StreamingCubeBuilder.java    |  49 +--
 .../engine/streaming/util/StreamingUtils.java   |  49 +--
 pom.xml                                         |   6 +-
 .../apache/kylin/source/hive/SchemaChecker.java |  18 +
 .../kylin/source/kafka/KafkaConfigManager.java  |  48 +--
 .../kylin/source/kafka/KafkaStreamingInput.java |  49 +--
 .../kylin/source/kafka/StreamingParser.java     |  48 +--
 .../source/kafka/StringStreamingParser.java     |  48 +--
 .../source/kafka/TimedJsonStreamParser.java     |  48 +--
 .../apache/kylin/source/kafka/TopicMeta.java    |  48 +--
 .../kylin/source/kafka/config/BrokerConfig.java |  48 +--
 .../kylin/source/kafka/config/KafkaConfig.java  |  48 +--
 .../kylin/source/kafka/util/KafkaRequester.java |  48 +--
 src/.settings/org.eclipse.core.resources.prefs  |   6 -
 src/.settings/org.eclipse.jdt.core.prefs        | 386 -------------------
 src/.settings/org.eclipse.jdt.ui.prefs          |   7 -
 src/main/config/assemblies/source-assembly.xml  | 117 ------
 .../checkstyle/checkstyle-java-header.txt       |  17 -
 src/main/config/checkstyle/checkstyle.xml       |  44 ---
 src/main/config/checkstyle/suppressions.xml     |  32 --
 .../storage/hbase/steps/HBaseCuboidWriter.java  |  49 +--
 51 files changed, 788 insertions(+), 1558 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/assembly/src/main/config/assemblies/source-assembly.xml
----------------------------------------------------------------------
diff --git a/assembly/src/main/config/assemblies/source-assembly.xml b/assembly/src/main/config/assemblies/source-assembly.xml
new file mode 100644
index 0000000..23a215b
--- /dev/null
+++ b/assembly/src/main/config/assemblies/source-assembly.xml
@@ -0,0 +1,117 @@
+<?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.
+-->
+<assembly>
+    <id>src</id>
+    <formats>
+        <format>zip</format>
+        <format>tar.gz</format>
+    </formats>
+    
+    <fileSets>
+        <!-- main project directory structure -->
+        <fileSet>
+            <directory>.</directory>
+            <outputDirectory>.</outputDirectory>
+            <useDefaultExcludes>true</useDefaultExcludes>
+            <excludes>
+                <!-- build output -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/).*${project.build.directory}.*]
+                </exclude>
+
+                <!-- NOTE: Most of the following excludes should not be required
+                  if the standard release process is followed. This is because the release
+                  plugin checks out project sources into a location like target/checkout, then
+                  runs the build from there. The result is a source-release archive that comes
+                  from a pretty clean directory structure. HOWEVER, if the release plugin is
+                  configured to run extra goals or generate a project website, it's definitely
+                  possible that some of these files will be present. So, it's safer to exclude
+                  them. -->
+
+                <!-- IDEs -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?maven-eclipse\.xml]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.project]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.classpath]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iws]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.idea(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?out(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.ipr]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iml]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.settings(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.externalToolBuilders(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.deployables(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.wtpmodules(/.*)?]
+                </exclude>
+
+
+                <!-- scm -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.gitignore(/.*)?]
+                </exclude>
+
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?docs/website(/.*)?]
+                </exclude>
+
+                <!-- release-plugin temp files -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.releaseBackup]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.next]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.tag]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]
+                </exclude>
+
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]
+                </exclude>
+
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?dist(/.*)?]
+                </exclude>
+
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?build(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?lib(/.*)?]
+                </exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?docs(/.*)?]
+                </exclude>
+            </excludes>
+        </fileSet>
+        <!-- LICENSE, NOTICE, DEPENDENCIES, git.properties, etc. calculated at build time -->
+        <fileSet>
+            <directory>${project.build.directory}/maven-shared-archive-resources/META-INF
+            </directory>
+            <outputDirectory>.</outputDirectory>
+        </fileSet>
+        <fileSet>
+            <directory>${project.build.directory}</directory>
+            <includes>
+                <include>git.properties</include>
+            </includes>
+            <outputDirectory>.</outputDirectory>
+        </fileSet>
+    </fileSets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/checkstyle-suppressions.xml b/checkstyle-suppressions.xml
deleted file mode 100644
index 06de625..0000000
--- a/checkstyle-suppressions.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0"?>
-<!DOCTYPE suppressions PUBLIC
-        "-//Puppy Crawl//DTD Suppressions 1.0//EN"
-        "http://www.puppycrawl.com/dtds/suppressions_1_0.dtd">
-<!--
-/**
- * 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.
- */
--->
-<suppressions>
-    <suppress checks="." files=".*[/\\]generated[/\\].*\.java"/>
-    <suppress checks="." files=".*[/\\]org[/\\]apache[/\\]calcite[/\\].*\.java"/>
-    <suppress checks="MagicNumberCheck" files=".*Test\.java"/>
-</suppressions>

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/checkstyle.xml
----------------------------------------------------------------------
diff --git a/checkstyle.xml b/checkstyle.xml
deleted file mode 100644
index 3c5b83f..0000000
--- a/checkstyle.xml
+++ /dev/null
@@ -1,62 +0,0 @@
-<?xml version="1.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.
-  -->
-
-<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.3//EN"
-        "http://www.puppycrawl.com/dtds/configuration_1_3.dtd">
-<module name="Checker">
-    <module name="FileTabCharacter"/>
-    <module name="SuppressWarningsFilter"/>
-    <module name="TreeWalker">
-        <module name="AvoidStarImport"/>
-        <module name="RedundantImport"/>
-        <module name="UnusedImports">
-            <property name="processJavadoc" value="true"/>
-        </module>
-        <module name="LineLength">
-            <property name="max" value="1000"/>
-            <property name="ignorePattern"
-                      value="^package.*|^import.*|a href|href|http://|https://|ftp://"/>
-        </module>
-        <module name="MethodLength"/>
-        <module name="MethodParamPad"/>
-        <module name="ParenPad"/>
-        <module name="EmptyStatement"/>
-        <module name="EmptyBlock">
-            <property name="option" value="text"/>
-        </module>
-        <module name="EqualsHashCode"/>
-        <module name="IllegalInstantiation"/>
-        <!--<module name="InnerAssignment"/>-->
-        <module name="MissingSwitchDefault"/>
-        <!--<module name="FinalClass"/>-->
-        <!--<module name="HideUtilityClassConstructor"/>-->
-        <!--<module name="InterfaceIsType"/>-->
-        <!--<module name="VisibilityModifier"/>-->
-        <module name="ArrayTypeStyle"/>
-        <module name="LeftCurly"/>
-        <module name="RightCurly"/>
-        <module name="UpperEll"/>
-        <!--<module name="NoFinalizer"/>-->
-        <!--<module name="MissingDeprecated"/>-->
-        <module name="WhitespaceAfter">
-            <property name="tokens" value="COMMA"/>
-        </module>
-        <module name="SuppressWarningsHolder"/>
-    </module>
-</module>

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-common/src/main/java/org/apache/kylin/common/KylinVersion.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinVersion.java b/core-common/src/main/java/org/apache/kylin/common/KylinVersion.java
index 6aa773c..4a55ae8 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinVersion.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinVersion.java
@@ -1,19 +1,21 @@
-/**
- * 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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+/*
+ * 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.common;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java b/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java
index a24cae6..f64f4b2 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/Bytes.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -14,7 +14,8 @@
  * 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.common.util;
 
 import static com.google.common.base.Preconditions.checkArgument;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-common/src/main/java/org/apache/kylin/common/util/MemoryBudgetController.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/MemoryBudgetController.java b/core-common/src/main/java/org/apache/kylin/common/util/MemoryBudgetController.java
index fec045e..920f8e6 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/MemoryBudgetController.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/MemoryBudgetController.java
@@ -1,19 +1,20 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.common.util;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-common/src/main/java/org/apache/kylin/common/util/Pair.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/Pair.java b/core-common/src/main/java/org/apache/kylin/common/util/Pair.java
index 9e4e9ee..8ea009a 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/Pair.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/Pair.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
  * 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
@@ -15,7 +14,7 @@
  * 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.common.util;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-common/src/main/java/org/apache/kylin/common/util/StreamingBatch.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/StreamingBatch.java b/core-common/src/main/java/org/apache/kylin/common/util/StreamingBatch.java
index ae5b72f..e000aa6 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/StreamingBatch.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/StreamingBatch.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.common.util;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
index 392a8ed..86c44e9 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/TrimmedCubeCodeSystem.java
@@ -1,22 +1,20 @@
 /*
+ * 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
  *
- *  * 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.
- * /
- */
+ *     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.gridtable;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
index 0bfaab3..651203a 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
@@ -1,19 +1,21 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.inmemcubing;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/CompoundCuboidWriter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/CompoundCuboidWriter.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/CompoundCuboidWriter.java
index 2eb748b..c82f418 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/CompoundCuboidWriter.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/CompoundCuboidWriter.java
@@ -1,19 +1,20 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.inmemcubing;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java
index 2af8bce..5219ede 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ConcurrentDiskStore.java
@@ -1,19 +1,20 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.inmemcubing;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
index 69f1f82..413c448 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/DoggedCubeBuilder.java
@@ -1,19 +1,21 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.inmemcubing;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java
index 9e24ec7..3f6cb0c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/ICuboidWriter.java
@@ -1,19 +1,20 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.inmemcubing;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
index e4908b8..66a444e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
@@ -1,19 +1,21 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.inmemcubing;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java
index d8cd814..3cfaed7 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/MemDiskStore.java
@@ -1,19 +1,20 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.inmemcubing;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
----------------------------------------------------------------------
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 aa4610f..61d30a0 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
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.util;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStorage.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStorage.java b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStorage.java
index ff95743..309e01d 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStorage.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/IGTStorage.java
@@ -1,22 +1,20 @@
 /*
+ * 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
  *
- *  * 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.
- * /
- */
+ *     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.gridtable;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java b/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java
index bb7503a..a0472e5 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/ScannerWorker.java
@@ -1,22 +1,20 @@
 /*
+ * 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
  *
- *  * 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.
- * /
- */
+ *     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.gridtable;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
index 3ec1d74..3396fd2 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
@@ -1,19 +1,20 @@
 /*
- *  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
+ * 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
+ *     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.
- */
+ * 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.gridtable;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/dev-support/checkstyle-java-header.txt
----------------------------------------------------------------------
diff --git a/dev-support/checkstyle-java-header.txt b/dev-support/checkstyle-java-header.txt
new file mode 100644
index 0000000..3740126
--- /dev/null
+++ b/dev-support/checkstyle-java-header.txt
@@ -0,0 +1,17 @@
+/*
+ * 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.
+*/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/dev-support/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/dev-support/checkstyle-suppressions.xml b/dev-support/checkstyle-suppressions.xml
new file mode 100644
index 0000000..06de625
--- /dev/null
+++ b/dev-support/checkstyle-suppressions.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0"?>
+<!DOCTYPE suppressions PUBLIC
+        "-//Puppy Crawl//DTD Suppressions 1.0//EN"
+        "http://www.puppycrawl.com/dtds/suppressions_1_0.dtd">
+<!--
+/**
+ * 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.
+ */
+-->
+<suppressions>
+    <suppress checks="." files=".*[/\\]generated[/\\].*\.java"/>
+    <suppress checks="." files=".*[/\\]org[/\\]apache[/\\]calcite[/\\].*\.java"/>
+    <suppress checks="MagicNumberCheck" files=".*Test\.java"/>
+</suppressions>

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/dev-support/checkstyle.xml
----------------------------------------------------------------------
diff --git a/dev-support/checkstyle.xml b/dev-support/checkstyle.xml
new file mode 100644
index 0000000..ec36760
--- /dev/null
+++ b/dev-support/checkstyle.xml
@@ -0,0 +1,67 @@
+<?xml version="1.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.
+  -->
+
+<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.3//EN"
+        "http://www.puppycrawl.com/dtds/configuration_1_3.dtd">
+<module name="Checker">
+    <module name="Header">
+        <property name="headerFile" value="dev-support/checkstyle-java-header.txt"/>
+        <property name="fileExtensions" value="java"/>
+        <property name="ignoreLines" value="9, 11, 17"/>
+    </module>
+    <module name="FileTabCharacter"/>
+    <module name="SuppressWarningsFilter"/>
+    <module name="TreeWalker">
+        <module name="AvoidStarImport"/>
+        <module name="RedundantImport"/>
+        <module name="UnusedImports">
+            <property name="processJavadoc" value="true"/>
+        </module>
+        <module name="LineLength">
+            <property name="max" value="1000"/>
+            <property name="ignorePattern"
+                      value="^package.*|^import.*|a href|href|http://|https://|ftp://"/>
+        </module>
+        <module name="MethodLength"/>
+        <module name="MethodParamPad"/>
+        <module name="ParenPad"/>
+        <module name="EmptyStatement"/>
+        <module name="EmptyBlock">
+            <property name="option" value="text"/>
+        </module>
+        <module name="EqualsHashCode"/>
+        <module name="IllegalInstantiation"/>
+        <!--<module name="InnerAssignment"/>-->
+        <module name="MissingSwitchDefault"/>
+        <!--<module name="FinalClass"/>-->
+        <!--<module name="HideUtilityClassConstructor"/>-->
+        <!--<module name="InterfaceIsType"/>-->
+        <!--<module name="VisibilityModifier"/>-->
+        <module name="ArrayTypeStyle"/>
+        <module name="LeftCurly"/>
+        <module name="RightCurly"/>
+        <module name="UpperEll"/>
+        <!--<module name="NoFinalizer"/>-->
+        <!--<module name="MissingDeprecated"/>-->
+        <module name="WhitespaceAfter">
+            <property name="tokens" value="COMMA"/>
+        </module>
+        <module name="SuppressWarningsHolder"/>
+    </module>
+</module>

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingInput.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingInput.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingInput.java
index 4b4cf02..c583283 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingInput.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingInput.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming;
 
 import org.apache.kylin.common.util.StreamingBatch;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingOutput.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingOutput.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingOutput.java
index 3fceccd..cb15e2b 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingOutput.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/IStreamingOutput.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming;
 
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java
index 9f9a4f4..c9da46e 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/OneOffStreamingBuilder.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming;
 
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java
index 7155625..8b0b8e6 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingBatchBuilder.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming;
 
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingConfig.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingConfig.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingConfig.java
index c8d1911..9d1a0b1 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingConfig.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingConfig.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java
index 7bf6ca2..271bf41 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingManager.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/StreamingCLI.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/StreamingCLI.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/StreamingCLI.java
index b1ec232..1d66b41 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/StreamingCLI.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/StreamingCLI.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming.cli;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java
index 180f0b8..e071a72 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cube/StreamingCubeBuilder.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming.cube;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/util/StreamingUtils.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/util/StreamingUtils.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/util/StreamingUtils.java
index 4038e53..5790bc1 100644
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/util/StreamingUtils.java
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/util/StreamingUtils.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.engine.streaming.util;
 
 import org.apache.kylin.common.util.ClassUtil;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index fbb27a4..1fdf81a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -940,8 +940,8 @@
                             <id>validate</id>
                             <phase>validate</phase>
                             <configuration>
-                                <configLocation>checkstyle.xml</configLocation>
-                                <suppressionsLocation>checkstyle-suppressions.xml</suppressionsLocation>
+                                <configLocation>dev-support/checkstyle.xml</configLocation>
+                                <suppressionsLocation>dev-support/checkstyle-suppressions.xml</suppressionsLocation>
                                 <encoding>UTF-8</encoding>
                                 <consoleOutput>true</consoleOutput>
                                 <failsOnError>true</failsOnError>
@@ -1109,7 +1109,7 @@
                                     <tarLongFileMode>posix</tarLongFileMode>
                                     <runOnlyAtExecutionRoot>true</runOnlyAtExecutionRoot>
                                     <appendAssemblyId>true</appendAssemblyId>
-                                    <descriptor>src/main/config/assemblies/source-assembly.xml</descriptor>
+                                    <descriptor>assembly/src/main/config/assemblies/source-assembly.xml</descriptor>
                                     <finalName>apache-kylin-${project.version}</finalName>
                                 </configuration>
                             </execution>

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java b/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
index 3b03551..e955410 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
@@ -1,3 +1,21 @@
+/*
+ * 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.source.hive;
 
 import static com.google.common.base.Preconditions.checkNotNull;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java
index 2de8527..e76422c 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaConfigManager.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaStreamingInput.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaStreamingInput.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaStreamingInput.java
index c3bdb75..66142c5 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaStreamingInput.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaStreamingInput.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
index cb6a72b..4d840b8 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka;
 


[23/50] [abbrv] kylin git commit: minor add jvm Xss size

Posted by li...@apache.org.
minor add jvm Xss size


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 4c9b67fa449d9b43cfd1cdf6fc19710bb368005f
Parents: c2ff8c5
Author: Jason <ji...@163.com>
Authored: Thu Sep 22 18:17:43 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Thu Sep 22 18:17:43 2016 +0800

----------------------------------------------------------------------
 build/bin/setenv.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4c9b67fa/build/bin/setenv.sh
----------------------------------------------------------------------
diff --git a/build/bin/setenv.sh b/build/bin/setenv.sh
index 317005b..24084f3 100755
--- a/build/bin/setenv.sh
+++ b/build/bin/setenv.sh
@@ -19,7 +19,7 @@
 
 # (if your're deploying KYLIN on a powerful server and want to replace the default conservative settings)
 # uncomment following to for it to take effect
-export KYLIN_JVM_SETTINGS="-Xms1024M -Xmx4096M -Xss256K -XX:MaxPermSize=128M -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:$KYLIN_HOME/logs/kylin.gc.$$ -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=64M"
+export KYLIN_JVM_SETTINGS="-Xms1024M -Xmx4096M -Xss1024K -XX:MaxPermSize=128M -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:$KYLIN_HOME/logs/kylin.gc.$$ -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=64M"
 # export KYLIN_JVM_SETTINGS="-Xms16g -Xmx16g -XX:MaxPermSize=512m -XX:NewSize=3g -XX:MaxNewSize=3g -XX:SurvivorRatio=4 -XX:+CMSClassUnloadingEnabled -XX:+CMSParallelRemarkEnabled -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -XX:CMSInitiatingOccupancyFraction=70 -XX:+DisableExplicitGC -XX:+HeapDumpOnOutOfMemoryError"
 
 # uncomment following to for it to take effect(the values need adjusting to fit your env)


[48/50] [abbrv] kylin git commit: KYLIN-2041 fix IT

Posted by li...@apache.org.
KYLIN-2041 fix IT


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 05afca7a8346370a2f507c4b477b46cbe4e258f7
Parents: 8596af1
Author: Yang Li <li...@apache.org>
Authored: Tue Sep 27 06:46:18 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Tue Sep 27 06:46:18 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java | 4 ++--
 .../java/org/apache/kylin/storage/hbase/HBaseResourceStore.java  | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/05afca7a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
index 5a2d5dd..618488f 100644
--- a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
@@ -57,9 +57,9 @@ public class ITHBaseResourceStoreTest extends HBaseMetadataTestCase {
         String path = "/cube/_test_large_cell.json";
         String largeContent = "THIS_IS_A_LARGE_CELL";
         StringEntity content = new StringEntity(largeContent);
-        Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
+        HBaseResourceStore store = (HBaseResourceStore) ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
+        Configuration hconf = store.getConnection().getConfiguration();
         int origSize = Integer.parseInt(hconf.get("hbase.client.keyvalue.maxsize", "10485760"));
-        ResourceStore store = ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
 
         try {
             hconf.set("hbase.client.keyvalue.maxsize", String.valueOf(largeContent.length() - 1));

http://git-wip-us.apache.org/repos/asf/kylin/blob/05afca7a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
index ee4b7e4..3fd6426 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
@@ -69,7 +69,7 @@ public class HBaseResourceStore extends ResourceStore {
     final String tableNameBase;
     final String hbaseUrl;
 
-    private HConnection getConnection() throws IOException {
+    HConnection getConnection() throws IOException {
         return HBaseConnection.get(hbaseUrl);
     }
 


[17/50] [abbrv] kylin git commit: KYLIN-2010 fix UT

Posted by li...@apache.org.
KYLIN-2010 fix UT


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: c1fc72386f72b9a86c547e612c6be8093dba4b68
Parents: 8babee8
Author: Yang Li <li...@apache.org>
Authored: Wed Sep 21 07:16:44 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Wed Sep 21 07:16:44 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java    | 2 +-
 .../src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java | 4 ++--
 .../main/java/org/apache/kylin/metadata/datatype/DataType.java   | 4 ++++
 3 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/c1fc7238/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
index e31111d..21e01b9 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
@@ -65,7 +65,7 @@ public class SegmentGTStartAndEnd {
         DataType partitionColType = info.getColumnType(index);
         if (partitionColType.isDate()) {
             value = DateFormat.formatToDateStr(ts);
-        } else if (partitionColType.isDatetime() || partitionColType.isTimestamp()) {
+        } else if (partitionColType.isTimeFamily()) {
             value = DateFormat.formatToTimeWithoutMilliStr(ts);
         } else if (partitionColType.isStringFamily()) {
             String partitionDateFormat = segment.getModel().getPartitionDesc().getPartitionDateFormat();

http://git-wip-us.apache.org/repos/asf/kylin/blob/c1fc7238/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
index a1d4440..12c4dfc 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
@@ -82,13 +82,13 @@ public class RowKeyColDesc {
             if (type.isDate()) {
                 encoding = encodingName = DateDimEnc.ENCODING_NAME;
             }
-            if (type.isTime() || type.isTimestamp() || type.isDatetime()) {
+            if (type.isTimeFamily()) {
                 encoding = encodingName = TimeDimEnc.ENCODING_NAME;
             }
         }
         if (DateDimEnc.ENCODING_NAME.equals(encodingName) && type.isDate() == false)
             throw new IllegalArgumentException(colRef + " type is " + type + " and cannot apply date encoding");
-        if (TimeDimEnc.ENCODING_NAME.equals(encodingName) && type.isTime() == false)
+        if (TimeDimEnc.ENCODING_NAME.equals(encodingName) && type.isTimeFamily() == false)
             throw new IllegalArgumentException(colRef + " type is " + type + " and cannot apply time encoding");
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/c1fc7238/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java
index c981f46..b726c5f 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataType.java
@@ -212,6 +212,10 @@ public class DataType implements Serializable {
         return DATETIME_FAMILY.contains(name);
     }
 
+    public boolean isTimeFamily() {
+        return DATETIME_FAMILY.contains(name) && !isDate();
+    }
+    
     public boolean isDate() {
         return name.equals("date");
     }


[02/50] [abbrv] kylin git commit: Revert "KYLIN-1726 use segment uuid instead of name"

Posted by li...@apache.org.
Revert "KYLIN-1726 use segment uuid instead of name"

This reverts commit 42dafc15db40731582d6257c618eff29643930a8.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 1f4880479cd3132786062723ba70312440de4805
Parents: dee8f2d
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:51:57 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/cube/CubeManager.java |  5 +---
 .../kylin/provision/BuildCubeWithStream.java    | 26 +++-----------------
 .../apache/kylin/source/kafka/KafkaMRInput.java |  2 +-
 .../source/kafka/hadoop/KafkaFlatTableJob.java  | 11 ++++++---
 .../kafka/hadoop/KafkaInputRecordReader.java    |  9 +++----
 5 files changed, 17 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/1f488047/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
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 fc68798..daeca0d 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -444,11 +444,8 @@ public class CubeManager implements IRealizationProvider {
         updateCube(cubeBuilder);
         return newSegment;
     }
-    public CubeSegment refreshSegment(CubeInstance cube, long startDate, long endDate, long startOffset, long endOffset) throws IOException {
-        return refreshSegment(cube, startDate, endDate, startOffset, endOffset, true);
-    }
 
-    public CubeSegment refreshSegment(CubeInstance cube, long startDate, long endDate, long startOffset, long endOffset, boolean strictChecking) throws IOException {
+    public CubeSegment refreshSegment(CubeInstance cube, long startDate, long endDate, long startOffset, long endOffset) throws IOException {
         checkNoBuildingSegment(cube);
 
         CubeSegment newSegment = newSegment(cube, startDate, endDate, startOffset, endOffset);

http://git-wip-us.apache.org/repos/asf/kylin/blob/1f488047/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index 9e779ab..7f79acc 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -21,8 +21,6 @@ package org.apache.kylin.provision;
 import java.io.File;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
-import java.util.HashMap;
-import java.util.List;
 import java.util.TimeZone;
 import java.util.UUID;
 
@@ -147,34 +145,18 @@ public class BuildCubeWithStream {
         //merge
         mergeSegment(cubeName, 0, 15000);
 
-        List<CubeSegment> segments = cubeManager.getCube(cubeName).getSegments();
-        Assert.assertTrue(segments.size() == 1);
-
-        CubeSegment toRefreshSeg = segments.get(0);
-        HashMap<String, String> partitionOffsetMap = toRefreshSeg.getAdditionalInfo();
-
-        refreshSegment(cubeName, toRefreshSeg.getSourceOffsetStart(), toRefreshSeg.getSourceOffsetEnd(), partitionOffsetMap);
-        segments = cubeManager.getCube(cubeName).getSegments();
-        Assert.assertTrue(segments.size() == 1);
-
     }
 
     private String mergeSegment(String cubeName, long startOffset, long endOffset) throws Exception {
-        CubeSegment segment = cubeManager.mergeSegments(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, false);
+        CubeSegment segment = cubeManager.mergeSegments(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, true);
         DefaultChainedExecutable job = EngineFactory.createBatchMergeJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());
         return job.getId();
     }
 
-    private String refreshSegment(String cubeName, long startOffset, long endOffset, HashMap<String, String> partitionOffsetMap) throws Exception {
-        CubeSegment segment = cubeManager.refreshSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, false);
-        segment.setAdditionalInfo(partitionOffsetMap);
-        CubeInstance cubeInstance = cubeManager.getCube(cubeName);
-        CubeUpdate cubeBuilder = new CubeUpdate(cubeInstance);
-        cubeBuilder.setToUpdateSegs(segment);
-        cubeManager.updateCube(cubeBuilder);
-        segment = cubeManager.getCube(cubeName).getSegmentById(segment.getUuid());
+    private String refreshSegment(String cubeName, long startOffset, long endOffset) throws Exception {
+        CubeSegment segment = cubeManager.refreshSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset);
         DefaultChainedExecutable job = EngineFactory.createBatchCubingJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());
@@ -182,7 +164,7 @@ public class BuildCubeWithStream {
     }
 
     private String buildSegment(String cubeName, long startOffset, long endOffset) throws Exception {
-        CubeSegment segment = cubeManager.appendSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, false);
+        CubeSegment segment = cubeManager.appendSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset);
         DefaultChainedExecutable job = EngineFactory.createBatchCubingJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());

http://git-wip-us.apache.org/repos/asf/kylin/blob/1f488047/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
index a5f678f..cfce137 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
@@ -165,7 +165,7 @@ public class KafkaMRInput implements IMRInput {
             jobBuilderSupport.appendMapReduceParameters(cmd);
             JobBuilderSupport.appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
             JobBuilderSupport.appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
-            JobBuilderSupport.appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
+            JobBuilderSupport.appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_NAME, seg.getName());
             JobBuilderSupport.appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_Save_Kafka_Data_" + seg.getRealization().getName() + "_Step");
 
             result.setMapReduceParams(cmd.toString());

http://git-wip-us.apache.org/repos/asf/kylin/blob/1f488047/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
index 87d2471..decfb60 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
@@ -33,6 +33,7 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
 import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.source.kafka.KafkaConfigManager;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
@@ -69,14 +70,14 @@ public class KafkaFlatTableJob extends AbstractHadoopJob {
             options.addOption(OPTION_JOB_NAME);
             options.addOption(OPTION_CUBE_NAME);
             options.addOption(OPTION_OUTPUT_PATH);
-            options.addOption(OPTION_SEGMENT_ID);
+            options.addOption(OPTION_SEGMENT_NAME);
             parseOptions(options, args);
 
             job = Job.getInstance(getConf(), getOptionValue(OPTION_JOB_NAME));
             String cubeName = getOptionValue(OPTION_CUBE_NAME);
             Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
 
-            String segmentId = getOptionValue(OPTION_SEGMENT_ID);
+            String segmentName = getOptionValue(OPTION_SEGMENT_NAME);
 
             // ----------------------------------------------------------------------------
             // add metadata to distributed cache
@@ -84,7 +85,7 @@ public class KafkaFlatTableJob extends AbstractHadoopJob {
             CubeInstance cube = cubeMgr.getCube(cubeName);
 
             job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
-            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_ID, segmentId);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
             logger.info("Starting: " + job.getJobName());
 
             setJobClasspath(job, cube.getConfig());
@@ -103,9 +104,11 @@ public class KafkaFlatTableJob extends AbstractHadoopJob {
             job.getConfiguration().set(CONFIG_KAFKA_TIMEOUT, String.valueOf(kafkaConfig.getTimeout()));
             job.getConfiguration().set(CONFIG_KAFKA_BUFFER_SIZE, String.valueOf(kafkaConfig.getBufferSize()));
             job.getConfiguration().set(CONFIG_KAFKA_INPUT_FORMAT, "json");
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
             job.getConfiguration().set(CONFIG_KAFKA_PARSER_NAME, kafkaConfig.getParserName());
             job.getConfiguration().set(CONFIG_KAFKA_CONSUMER_GROUP, cubeName); // use cubeName as consumer group name
-            setupMapper(cube.getSegmentById(segmentId));
+            setupMapper(cube.getSegment(segmentName, SegmentStatusEnum.NEW));
             job.setNumReduceTasks(0);
             FileOutputFormat.setOutputPath(job, output);
             FileOutputFormat.setCompressOutput(job, true);

http://git-wip-us.apache.org/repos/asf/kylin/blob/1f488047/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputRecordReader.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputRecordReader.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputRecordReader.java
index 6774c9d..f67fef5 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputRecordReader.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputRecordReader.java
@@ -105,11 +105,6 @@ public class KafkaInputRecordReader extends RecordReader<LongWritable, BytesWrit
             value = new BytesWritable();
         }
 
-        if (watermark >= latestOffset) {
-            log.info("Reach the end offset, stop reading.");
-            return false;
-        }
-
         if (messages == null) {
             log.info("{} fetching offset {} ", topic + ":" + split.getBrokers() + ":" + partition, watermark);
             TopicPartition topicPartition = new TopicPartition(topic, partition);
@@ -124,6 +119,10 @@ public class KafkaInputRecordReader extends RecordReader<LongWritable, BytesWrit
 
         if (iterator.hasNext()) {
             ConsumerRecord<String, String> message = iterator.next();
+            if (message.offset() >= latestOffset) {
+                log.info("Reach the end offset, stop reading.");
+                return false;
+            }
             key.set(message.offset());
             byte[] valuebytes = Bytes.toBytes(message.value());
             value.set(valuebytes, 0, valuebytes.length);


[20/50] [abbrv] kylin git commit: KYLIN-2010 filter encoding options for column in TOP_N measure

Posted by li...@apache.org.
KYLIN-2010 filter encoding options for column in TOP_N measure


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 32a9c67068edfa0778ab5e19fc8376813e1bbc90
Parents: 69cb49d
Author: Jason <ji...@163.com>
Authored: Wed Sep 21 15:25:08 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Wed Sep 21 15:25:08 2016 +0800

----------------------------------------------------------------------
 webapp/app/partials/cubeDesigner/measures.html | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/32a9c670/webapp/app/partials/cubeDesigner/measures.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/measures.html b/webapp/app/partials/cubeDesigner/measures.html
index 56c86be..ec7110f 100755
--- a/webapp/app/partials/cubeDesigner/measures.html
+++ b/webapp/app/partials/cubeDesigner/measures.html
@@ -260,7 +260,7 @@
                                       <select ng-if="state.mode=='edit'" style="width:180px;"
                                               chosen ng-model="groupby_column.encoding"
                                               ng-change="refreshGroupBy(convertedColumns,$index,groupby_column)"
-                                              ng-options="dt as dt for dt in store.supportedEncoding">
+                                              ng-options="dt.value as dt.name for dt in getEncodings(groupby_column.name)">
                                         <option value=""></option>
                                       </select>
                                       <span ng-if="state.mode=='view'">{{groupby_column.encoding}}</span>


[16/50] [abbrv] kylin git commit: deprecate only IntegerDimEnc and rename SlimLongDimEnc to IntegerDimEnc

Posted by li...@apache.org.
deprecate only IntegerDimEnc and rename SlimLongDimEnc to IntegerDimEnc


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 8babee8d09f48d8ed60c70f16258402822ea81bf
Parents: f0d2be3
Author: Yang Li <li...@apache.org>
Authored: Wed Sep 21 06:53:10 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Wed Sep 21 06:53:10 2016 +0800

----------------------------------------------------------------------
 .../gridtable/DimEncodingPreserveOrderTest.java |   6 +-
 .../dimension/DimensionEncodingFactory.java     |   2 +-
 .../org/apache/kylin/dimension/IntDimEnc.java   | 197 +++++++++++++++++
 .../apache/kylin/dimension/IntegerDimEnc.java   |  39 +++-
 .../apache/kylin/dimension/SlimLongDimEnc.java  | 219 -------------------
 .../apache/kylin/dimension/IntDimEncTest.java   | 131 +++++++++++
 .../kylin/dimension/IntegerDimEncTest.java      |  53 +++--
 .../kylin/dimension/SlimLongDimEncTest.java     | 151 -------------
 8 files changed, 400 insertions(+), 398 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8babee8d/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
----------------------------------------------------------------------
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 0c84ed0..ffd43e5 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
@@ -27,7 +27,7 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.dimension.DimensionEncoding;
 import org.apache.kylin.dimension.FixedLenHexDimEnc;
 import org.apache.kylin.dimension.OneMoreByteVLongDimEnc;
-import org.apache.kylin.dimension.SlimLongDimEnc;
+import org.apache.kylin.dimension.IntegerDimEnc;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
@@ -52,7 +52,7 @@ public class DimEncodingPreserveOrderTest {
 
     @Test
     public void testVLongDimEncPreserveOrder() {
-        SlimLongDimEnc enc = new SlimLongDimEnc(2);
+        IntegerDimEnc enc = new IntegerDimEnc(2);
         List<ByteArray> encodedValues = Lists.newArrayList();
         encodedValues.add(encode(enc, -32767L));
         encodedValues.add(encode(enc, -10000L));
@@ -68,7 +68,7 @@ public class DimEncodingPreserveOrderTest {
 
     @Test
     public void testVLongDimEncPreserveOrder2() {
-        SlimLongDimEnc enc = new SlimLongDimEnc(8);
+        IntegerDimEnc enc = new IntegerDimEnc(8);
         List<ByteArray> encodedValues = Lists.newArrayList();
         encodedValues.add(encode(enc, -Long.MAX_VALUE));
         encodedValues.add(encode(enc, -10000L));

http://git-wip-us.apache.org/repos/asf/kylin/blob/8babee8d/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java b/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
index 739fa25..27bebd7 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/DimensionEncodingFactory.java
@@ -74,8 +74,8 @@ public abstract class DimensionEncodingFactory {
 
             // built-in encodings, note dictionary is a special case
             map.put(FixedLenDimEnc.ENCODING_NAME, new FixedLenDimEnc.Factory());
+            map.put(IntDimEnc.ENCODING_NAME, new IntDimEnc.Factory());
             map.put(IntegerDimEnc.ENCODING_NAME, new IntegerDimEnc.Factory());
-            map.put(SlimLongDimEnc.ENCODING_NAME, new SlimLongDimEnc.Factory());
             map.put(FixedLenHexDimEnc.ENCODING_NAME, new FixedLenHexDimEnc.Factory());
             map.put(DateDimEnc.ENCODING_NAME, new DateDimEnc.Factory());
             map.put(TimeDimEnc.ENCODING_NAME, new TimeDimEnc.Factory());

http://git-wip-us.apache.org/repos/asf/kylin/blob/8babee8d/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java
new file mode 100644
index 0000000..88af716
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java
@@ -0,0 +1,197 @@
+/*
+ * 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.dimension;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * deprecated use IntegerDimEnc instead
+ * @deprecated
+ */
+public class IntDimEnc extends DimensionEncoding {
+    private static final long serialVersionUID = 1L;
+
+    private static Logger logger = LoggerFactory.getLogger(IntDimEnc.class);
+
+    private static final long[] CAP = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, Long.MAX_VALUE };
+
+    public static final String ENCODING_NAME = "int";
+
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new IntDimEnc(Integer.parseInt(args[0]));
+        }
+    };
+
+    // ============================================================================
+
+    private int fixedLen;
+
+    transient private int avoidVerbose = 0;
+
+    //no-arg constructor is required for Externalizable
+    public IntDimEnc() {
+    }
+
+    public IntDimEnc(int len) {
+        if (len <= 0 || len >= CAP.length)
+            throw new IllegalArgumentException();
+
+        this.fixedLen = len;
+    }
+
+    @Override
+    public int getLengthOfEncoding() {
+        return fixedLen;
+    }
+
+    @Override
+    public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
+        if (value == null) {
+            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            return;
+        }
+
+        encode(Bytes.toString(value, 0, valueLen), output, outputOffset);
+    }
+
+    void encode(String valueStr, byte[] output, int outputOffset) {
+        if (valueStr == null) {
+            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            return;
+        }
+
+        long integer = Long.parseLong(valueStr);
+        if (integer > CAP[fixedLen]) {
+            if (avoidVerbose++ % 10000 == 0) {
+                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
+            }
+        }
+
+        BytesUtil.writeLong(integer, output, outputOffset, fixedLen);
+    }
+
+    @Override
+    public String decode(byte[] bytes, int offset, int len) {
+        if (isNull(bytes, offset, len)) {
+            return null;
+        }
+
+        long integer = BytesUtil.readLong(bytes, offset, len);
+        return String.valueOf(integer);
+    }
+
+    @Override
+    public DataTypeSerializer<Object> asDataTypeSerializer() {
+        return new IntegerSerializer();
+    }
+
+    public class IntegerSerializer extends DataTypeSerializer<Object> {
+        // be thread-safe and avoid repeated obj creation
+        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
+
+        private byte[] currentBuf() {
+            byte[] buf = current.get();
+            if (buf == null) {
+                buf = new byte[fixedLen];
+                current.set(buf);
+            }
+            return buf;
+        }
+
+        @Override
+        public void serialize(Object value, ByteBuffer out) {
+            byte[] buf = currentBuf();
+            String valueStr = value == null ? null : value.toString();
+            encode(valueStr, buf, 0);
+            out.put(buf);
+        }
+
+        @Override
+        public Object deserialize(ByteBuffer in) {
+            byte[] buf = currentBuf();
+            in.get(buf);
+            return decode(buf, 0, buf.length);
+        }
+
+        @Override
+        public int peekLength(ByteBuffer in) {
+            return fixedLen;
+        }
+
+        @Override
+        public int maxLength() {
+            return fixedLen;
+        }
+
+        @Override
+        public int getStorageBytesEstimate() {
+            return fixedLen;
+        }
+
+        @Override
+        public Object valueOf(String str) {
+            return str;
+        }
+    }
+
+    @Override
+    public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeShort(fixedLen);
+    }
+
+    @Override
+    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        fixedLen = in.readShort();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        IntDimEnc that = (IntDimEnc) o;
+
+        return fixedLen == that.fixedLen;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return fixedLen;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8babee8d/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
index b87e46a..a8cb510 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
@@ -6,9 +6,9 @@
  * 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.
@@ -31,16 +31,24 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * deprecated use SlimLongDimEnc instead
+ * replacement for IntegerDimEnc, the diff is VLongDimEnc supports negative values
  */
 public class IntegerDimEnc extends DimensionEncoding {
     private static final long serialVersionUID = 1L;
 
     private static Logger logger = LoggerFactory.getLogger(IntegerDimEnc.class);
 
-    private static final long[] CAP = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, Long.MAX_VALUE };
+    private static final long[] CAP = { 0, 0x7fL, 0x7fffL, 0x7fffffL, 0x7fffffffL, 0x7fffffffffL, 0x7fffffffffffL, 0x7fffffffffffffL, 0x7fffffffffffffffL };
+    private static final long[] MASK = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, 0xffffffffffffffffL };
+    private static final long[] TAIL = { 0, 0x80L, 0x8000L, 0x800000L, 0x80000000L, 0x8000000000L, 0x800000000000L, 0x80000000000000L, 0x8000000000000000L };
+    static {
+        for (int i = 1; i < TAIL.length; ++i) {
+            long head = ~MASK[i];
+            TAIL[i] = head | TAIL[i];
+        }
+    }
 
-    public static final String ENCODING_NAME = "int";
+    public static final String ENCODING_NAME = "integer";
 
     public static class Factory extends DimensionEncodingFactory {
         @Override
@@ -59,6 +67,7 @@ public class IntegerDimEnc extends DimensionEncoding {
     private int fixedLen;
 
     transient private int avoidVerbose = 0;
+    transient private int avoidVerbose2 = 0;
 
     //no-arg constructor is required for Externalizable
     public IntegerDimEnc() {
@@ -93,13 +102,19 @@ public class IntegerDimEnc extends DimensionEncoding {
         }
 
         long integer = Long.parseLong(valueStr);
-        if (integer > CAP[fixedLen]) {
+        if (integer > CAP[fixedLen] || integer < TAIL[fixedLen]) {
             if (avoidVerbose++ % 10000 == 0) {
                 logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
             }
         }
 
-        BytesUtil.writeLong(integer, output, outputOffset, fixedLen);
+        if (integer == TAIL[fixedLen]) {
+            if (avoidVerbose2++ % 10000 == 0) {
+                logger.warn("Value " + valueStr + " does not fit into " + fixedLen + " bytes ");
+            }
+        }
+
+        BytesUtil.writeLong(integer + CAP[fixedLen], output, outputOffset, fixedLen);//apply an offset to preserve binary order, overflow is okay
     }
 
     @Override
@@ -108,7 +123,15 @@ public class IntegerDimEnc extends DimensionEncoding {
             return null;
         }
 
-        long integer = BytesUtil.readLong(bytes, offset, len);
+        long integer = BytesUtil.readLong(bytes, offset, len) - CAP[fixedLen];
+
+        //only take useful bytes
+        integer = integer & MASK[fixedLen];
+        boolean positive = (integer & ((0x80) << ((fixedLen - 1) << 3))) == 0;
+        if (!positive) {
+            integer |= (~MASK[fixedLen]);
+        }
+
         return String.valueOf(integer);
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8babee8d/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
deleted file mode 100644
index 4ac871f..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
+++ /dev/null
@@ -1,219 +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.dimension;
-
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataTypeSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * replacement for IntegerDimEnc, the diff is VLongDimEnc supports negative values
- */
-public class SlimLongDimEnc extends DimensionEncoding {
-    private static final long serialVersionUID = 1L;
-
-    private static Logger logger = LoggerFactory.getLogger(SlimLongDimEnc.class);
-
-    private static final long[] CAP = { 0, 0x7fL, 0x7fffL, 0x7fffffL, 0x7fffffffL, 0x7fffffffffL, 0x7fffffffffffL, 0x7fffffffffffffL, 0x7fffffffffffffffL };
-    private static final long[] MASK = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, 0xffffffffffffffffL };
-    private static final long[] TAIL = { 0, 0x80L, 0x8000L, 0x800000L, 0x80000000L, 0x8000000000L, 0x800000000000L, 0x80000000000000L, 0x8000000000000000L };
-    static {
-        for (int i = 1; i < TAIL.length; ++i) {
-            long head = ~MASK[i];
-            TAIL[i] = head | TAIL[i];
-        }
-    }
-
-    public static final String ENCODING_NAME = "slimlong";
-
-    public static class Factory extends DimensionEncodingFactory {
-        @Override
-        public String getSupportedEncodingName() {
-            return ENCODING_NAME;
-        }
-
-        @Override
-        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
-            return new SlimLongDimEnc(Integer.parseInt(args[0]));
-        }
-    };
-
-    // ============================================================================
-
-    private int fixedLen;
-
-    transient private int avoidVerbose = 0;
-    transient private int avoidVerbose2 = 0;
-
-    //no-arg constructor is required for Externalizable
-    public SlimLongDimEnc() {
-    }
-
-    public SlimLongDimEnc(int len) {
-        if (len <= 0 || len >= CAP.length)
-            throw new IllegalArgumentException();
-
-        this.fixedLen = len;
-    }
-
-    @Override
-    public int getLengthOfEncoding() {
-        return fixedLen;
-    }
-
-    @Override
-    public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
-        if (value == null) {
-            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
-            return;
-        }
-
-        encode(Bytes.toString(value, 0, valueLen), output, outputOffset);
-    }
-
-    void encode(String valueStr, byte[] output, int outputOffset) {
-        if (valueStr == null) {
-            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
-            return;
-        }
-
-        long integer = Long.parseLong(valueStr);
-        if (integer > CAP[fixedLen] || integer < TAIL[fixedLen]) {
-            if (avoidVerbose++ % 10000 == 0) {
-                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
-            }
-        }
-
-        if (integer == TAIL[fixedLen]) {
-            if (avoidVerbose2++ % 10000 == 0) {
-                logger.warn("Value " + valueStr + " does not fit into " + fixedLen + " bytes ");
-            }
-        }
-
-        BytesUtil.writeLong(integer + CAP[fixedLen], output, outputOffset, fixedLen);//apply an offset to preserve binary order, overflow is okay
-    }
-
-    @Override
-    public String decode(byte[] bytes, int offset, int len) {
-        if (isNull(bytes, offset, len)) {
-            return null;
-        }
-
-        long integer = BytesUtil.readLong(bytes, offset, len) - CAP[fixedLen];
-
-        //only take useful bytes
-        integer = integer & MASK[fixedLen];
-        boolean positive = (integer & ((0x80) << ((fixedLen - 1) << 3))) == 0;
-        if (!positive) {
-            integer |= (~MASK[fixedLen]);
-        }
-
-        return String.valueOf(integer);
-    }
-
-    @Override
-    public DataTypeSerializer<Object> asDataTypeSerializer() {
-        return new IntegerSerializer();
-    }
-
-    public class IntegerSerializer extends DataTypeSerializer<Object> {
-        // be thread-safe and avoid repeated obj creation
-        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
-
-        private byte[] currentBuf() {
-            byte[] buf = current.get();
-            if (buf == null) {
-                buf = new byte[fixedLen];
-                current.set(buf);
-            }
-            return buf;
-        }
-
-        @Override
-        public void serialize(Object value, ByteBuffer out) {
-            byte[] buf = currentBuf();
-            String valueStr = value == null ? null : value.toString();
-            encode(valueStr, buf, 0);
-            out.put(buf);
-        }
-
-        @Override
-        public Object deserialize(ByteBuffer in) {
-            byte[] buf = currentBuf();
-            in.get(buf);
-            return decode(buf, 0, buf.length);
-        }
-
-        @Override
-        public int peekLength(ByteBuffer in) {
-            return fixedLen;
-        }
-
-        @Override
-        public int maxLength() {
-            return fixedLen;
-        }
-
-        @Override
-        public int getStorageBytesEstimate() {
-            return fixedLen;
-        }
-
-        @Override
-        public Object valueOf(String str) {
-            return str;
-        }
-    }
-
-    @Override
-    public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeShort(fixedLen);
-    }
-
-    @Override
-    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        fixedLen = in.readShort();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        SlimLongDimEnc that = (SlimLongDimEnc) o;
-
-        return fixedLen == that.fixedLen;
-
-    }
-
-    @Override
-    public int hashCode() {
-        return fixedLen;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8babee8d/core-metadata/src/test/java/org/apache/kylin/dimension/IntDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/IntDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/IntDimEncTest.java
new file mode 100644
index 0000000..280a242
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/IntDimEncTest.java
@@ -0,0 +1,131 @@
+/*
+ * 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.dimension;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Deprecated. use VLongDimEnc instead
+ * @deprecated
+ */
+public class IntDimEncTest {
+
+    @Test
+    public void testConstructor() {
+        try {
+            new IntDimEnc(0);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        try {
+            new IntDimEnc(9);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        new IntDimEnc(8);
+    }
+
+    @Test
+    public void testNull() {
+        for (int i = 1; i < 9; i++) {
+            IntDimEnc enc = new IntDimEnc(i);
+
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            String decode = enc.decode(buf, 0, buf.length);
+            Assert.assertEquals(null, decode);
+
+            buf = new byte[enc.getLengthOfEncoding()];
+            DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+            ser.serialize(null, ByteBuffer.wrap(buf));
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+            Assert.assertEquals(null, decode);
+        }
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        IntDimEnc enc = new IntDimEnc(2);
+        testEncodeDecode(enc, 0);
+        testEncodeDecode(enc, 100);
+        testEncodeDecode(enc, 10000);
+        testEncodeDecode(enc, 65534);
+        try {
+            testEncodeDecode(enc, 65535);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<65535> but was:<null>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, 65536);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[65536]> but was:<[0]>", e.getMessage());
+        }
+    }
+
+    private void testEncodeDecode(IntDimEnc enc, long value) {
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = "" + value;
+        byte[] bytes = Bytes.toBytes(valueStr);
+        enc.encode(bytes, bytes.length, buf, 0);
+        String decode = enc.decode(buf, 0, buf.length);
+        Assert.assertEquals(valueStr, decode);
+    }
+
+    @Test
+    public void testSerDes() {
+        IntDimEnc enc = new IntDimEnc(2);
+        testSerDes(enc, 0);
+        testSerDes(enc, 100);
+        testSerDes(enc, 10000);
+        testSerDes(enc, 65534);
+        try {
+            testSerDes(enc, 65535);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<65535> but was:<null>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, 65536);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[65536]> but was:<[0]>", e.getMessage());
+        }
+    }
+
+    private void testSerDes(IntDimEnc enc, long value) {
+        DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = "" + value;
+        ser.serialize(valueStr, ByteBuffer.wrap(buf));
+        String decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+        Assert.assertEquals(valueStr, decode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8babee8d/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
index fccd8b9..1bdb1d0 100644
--- a/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
@@ -6,9 +6,9 @@
  * 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.
@@ -25,9 +25,6 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 import org.junit.Assert;
 import org.junit.Test;
 
-/**
- * Deprecated. use VLongDimEnc instead
- */
 public class IntegerDimEncTest {
 
     @Test
@@ -73,24 +70,45 @@ public class IntegerDimEncTest {
         testEncodeDecode(enc, 0);
         testEncodeDecode(enc, 100);
         testEncodeDecode(enc, 10000);
-        testEncodeDecode(enc, 65534);
+        testEncodeDecode(enc, 32767);
+        testEncodeDecode(enc, -100);
+        testEncodeDecode(enc, -10000);
+        testEncodeDecode(enc, -32767);
         try {
-            testEncodeDecode(enc, 65535);
+            testEncodeDecode(enc, 32768);
             Assert.fail();
         } catch (Throwable e) {
-            Assert.assertEquals("expected:<65535> but was:<null>", e.getMessage());
+            Assert.assertEquals("expected:<32768> but was:<null>", e.getMessage());
         }
         try {
-            testEncodeDecode(enc, 65536);
+            testEncodeDecode(enc, -32768);
             Assert.fail();
         } catch (Throwable e) {
-            Assert.assertEquals("expected:<[65536]> but was:<[0]>", e.getMessage());
+            Assert.assertEquals("expected:<-32768> but was:<null>", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testEncodeDecode2() {
+        IntegerDimEnc enc = new IntegerDimEnc(8);
+        testEncodeDecode(enc, 0);
+        testEncodeDecode(enc, 100);
+        testEncodeDecode(enc, 10000);
+        testEncodeDecode(enc, Long.MAX_VALUE);
+        testEncodeDecode(enc, -100);
+        testEncodeDecode(enc, -10000);
+        testEncodeDecode(enc, -Long.MAX_VALUE);
+        try {
+            testEncodeDecode(enc, Long.MIN_VALUE);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<-9223372036854775808> but was:<null>", e.getMessage());
         }
     }
 
     private void testEncodeDecode(IntegerDimEnc enc, long value) {
-        byte[] buf = new byte[enc.getLengthOfEncoding()];
         String valueStr = "" + value;
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
         byte[] bytes = Bytes.toBytes(valueStr);
         enc.encode(bytes, bytes.length, buf, 0);
         String decode = enc.decode(buf, 0, buf.length);
@@ -103,18 +121,21 @@ public class IntegerDimEncTest {
         testSerDes(enc, 0);
         testSerDes(enc, 100);
         testSerDes(enc, 10000);
-        testSerDes(enc, 65534);
+        testSerDes(enc, 32767);
+        testSerDes(enc, -100);
+        testSerDes(enc, -10000);
+        testSerDes(enc, -32767);
         try {
-            testSerDes(enc, 65535);
+            testSerDes(enc, 32768);
             Assert.fail();
         } catch (Throwable e) {
-            Assert.assertEquals("expected:<65535> but was:<null>", e.getMessage());
+            Assert.assertEquals("expected:<32768> but was:<null>", e.getMessage());
         }
         try {
-            testSerDes(enc, 65536);
+            testSerDes(enc, -32768);
             Assert.fail();
         } catch (Throwable e) {
-            Assert.assertEquals("expected:<[65536]> but was:<[0]>", e.getMessage());
+            Assert.assertEquals("expected:<-32768> but was:<null>", e.getMessage());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8babee8d/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
deleted file mode 100644
index a1e9516..0000000
--- a/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
+++ /dev/null
@@ -1,151 +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.dimension;
-
-import java.nio.ByteBuffer;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.metadata.datatype.DataTypeSerializer;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class SlimLongDimEncTest {
-
-    @Test
-    public void testConstructor() {
-        try {
-            new SlimLongDimEnc(0);
-            Assert.fail();
-        } catch (IllegalArgumentException e) {
-            // expect
-        }
-        try {
-            new SlimLongDimEnc(9);
-            Assert.fail();
-        } catch (IllegalArgumentException e) {
-            // expect
-        }
-        new SlimLongDimEnc(8);
-    }
-
-    @Test
-    public void testNull() {
-        for (int i = 1; i < 9; i++) {
-            SlimLongDimEnc enc = new SlimLongDimEnc(i);
-
-            byte[] buf = new byte[enc.getLengthOfEncoding()];
-            enc.encode(null, 0, buf, 0);
-            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
-            String decode = enc.decode(buf, 0, buf.length);
-            Assert.assertEquals(null, decode);
-
-            buf = new byte[enc.getLengthOfEncoding()];
-            DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
-            ser.serialize(null, ByteBuffer.wrap(buf));
-            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
-            decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
-            Assert.assertEquals(null, decode);
-        }
-    }
-
-    @Test
-    public void testEncodeDecode() {
-        SlimLongDimEnc enc = new SlimLongDimEnc(2);
-        testEncodeDecode(enc, 0);
-        testEncodeDecode(enc, 100);
-        testEncodeDecode(enc, 10000);
-        testEncodeDecode(enc, 32767);
-        testEncodeDecode(enc, -100);
-        testEncodeDecode(enc, -10000);
-        testEncodeDecode(enc, -32767);
-        try {
-            testEncodeDecode(enc, 32768);
-            Assert.fail();
-        } catch (Throwable e) {
-            Assert.assertEquals("expected:<32768> but was:<null>", e.getMessage());
-        }
-        try {
-            testEncodeDecode(enc, -32768);
-            Assert.fail();
-        } catch (Throwable e) {
-            Assert.assertEquals("expected:<-32768> but was:<null>", e.getMessage());
-        }
-    }
-
-    @Test
-    public void testEncodeDecode2() {
-        SlimLongDimEnc enc = new SlimLongDimEnc(8);
-        testEncodeDecode(enc, 0);
-        testEncodeDecode(enc, 100);
-        testEncodeDecode(enc, 10000);
-        testEncodeDecode(enc, Long.MAX_VALUE);
-        testEncodeDecode(enc, -100);
-        testEncodeDecode(enc, -10000);
-        testEncodeDecode(enc, -Long.MAX_VALUE);
-        try {
-            testEncodeDecode(enc, Long.MIN_VALUE);
-            Assert.fail();
-        } catch (Throwable e) {
-            Assert.assertEquals("expected:<-9223372036854775808> but was:<null>", e.getMessage());
-        }
-    }
-
-    private void testEncodeDecode(SlimLongDimEnc enc, long value) {
-        String valueStr = "" + value;
-        byte[] buf = new byte[enc.getLengthOfEncoding()];
-        byte[] bytes = Bytes.toBytes(valueStr);
-        enc.encode(bytes, bytes.length, buf, 0);
-        String decode = enc.decode(buf, 0, buf.length);
-        Assert.assertEquals(valueStr, decode);
-    }
-
-    @Test
-    public void testSerDes() {
-        SlimLongDimEnc enc = new SlimLongDimEnc(2);
-        testSerDes(enc, 0);
-        testSerDes(enc, 100);
-        testSerDes(enc, 10000);
-        testSerDes(enc, 32767);
-        testSerDes(enc, -100);
-        testSerDes(enc, -10000);
-        testSerDes(enc, -32767);
-        try {
-            testSerDes(enc, 32768);
-            Assert.fail();
-        } catch (Throwable e) {
-            Assert.assertEquals("expected:<32768> but was:<null>", e.getMessage());
-        }
-        try {
-            testSerDes(enc, -32768);
-            Assert.fail();
-        } catch (Throwable e) {
-            Assert.assertEquals("expected:<-32768> but was:<null>", e.getMessage());
-        }
-    }
-
-    private void testSerDes(SlimLongDimEnc enc, long value) {
-        DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
-        byte[] buf = new byte[enc.getLengthOfEncoding()];
-        String valueStr = "" + value;
-        ser.serialize(valueStr, ByteBuffer.wrap(buf));
-        String decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
-        Assert.assertEquals(valueStr, decode);
-    }
-
-}


[46/50] [abbrv] kylin git commit: KYLIN-2019: Enable Apache Licence checker for Checkstyle

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
index 8888d67..cea8e0b 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
index 20c57a9..d3530f1 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/TopicMeta.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/TopicMeta.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/TopicMeta.java
index 4145ef6..a73543e 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/TopicMeta.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/TopicMeta.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/BrokerConfig.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/BrokerConfig.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/BrokerConfig.java
index dbc5fd7..add825b 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/BrokerConfig.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/BrokerConfig.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka.config;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java
index 9c19588..c538acb 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaConfig.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka.config;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaRequester.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaRequester.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaRequester.java
index 58cba7d..919db20 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaRequester.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaRequester.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.source.kafka.util;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/src/.settings/org.eclipse.core.resources.prefs
----------------------------------------------------------------------
diff --git a/src/.settings/org.eclipse.core.resources.prefs b/src/.settings/org.eclipse.core.resources.prefs
deleted file mode 100644
index 29abf99..0000000
--- a/src/.settings/org.eclipse.core.resources.prefs
+++ /dev/null
@@ -1,6 +0,0 @@
-eclipse.preferences.version=1
-encoding//src/main/java=UTF-8
-encoding//src/main/resources=UTF-8
-encoding//src/test/java=UTF-8
-encoding//src/test/resources=UTF-8
-encoding/<project>=UTF-8

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/src/.settings/org.eclipse.jdt.core.prefs
----------------------------------------------------------------------
diff --git a/src/.settings/org.eclipse.jdt.core.prefs b/src/.settings/org.eclipse.jdt.core.prefs
deleted file mode 100644
index 5aaaf1e..0000000
--- a/src/.settings/org.eclipse.jdt.core.prefs
+++ /dev/null
@@ -1,386 +0,0 @@
-eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled
-org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore
-org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull
-org.eclipse.jdt.core.compiler.annotation.nonnull.secondary=
-org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
-org.eclipse.jdt.core.compiler.annotation.nonnullbydefault.secondary=
-org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
-org.eclipse.jdt.core.compiler.annotation.nullable.secondary=
-org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
-org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled
-org.eclipse.jdt.core.compiler.codegen.methodParameters=do not generate
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
-org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve
-org.eclipse.jdt.core.compiler.compliance=1.7
-org.eclipse.jdt.core.compiler.debug.lineNumber=generate
-org.eclipse.jdt.core.compiler.debug.localVariable=generate
-org.eclipse.jdt.core.compiler.debug.sourceFile=generate
-org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning
-org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
-org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
-org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning
-org.eclipse.jdt.core.compiler.problem.deadCode=warning
-org.eclipse.jdt.core.compiler.problem.deprecation=warning
-org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
-org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
-org.eclipse.jdt.core.compiler.problem.discouragedReference=ignore
-org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
-org.eclipse.jdt.core.compiler.problem.enumIdentifier=error
-org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore
-org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
-org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
-org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
-org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning
-org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning
-org.eclipse.jdt.core.compiler.problem.forbiddenReference=ignore
-org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning
-org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
-org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning
-org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning
-org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
-org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
-org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning
-org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
-org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
-org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
-org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
-org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
-org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled
-org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning
-org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
-org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning
-org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning
-org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
-org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning
-org.eclipse.jdt.core.compiler.problem.nonnullTypeVariableFromLegacyInvocation=warning
-org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
-org.eclipse.jdt.core.compiler.problem.nullReference=warning
-org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
-org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning
-org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning
-org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
-org.eclipse.jdt.core.compiler.problem.pessimisticNullAnalysisForFreeTypeVariables=warning
-org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
-org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
-org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore
-org.eclipse.jdt.core.compiler.problem.rawTypeReference=ignore
-org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning
-org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
-org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
-org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
-org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
-org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
-org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
-org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning
-org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled
-org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
-org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled
-org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
-org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning
-org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled
-org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=ignore
-org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning
-org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
-org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning
-org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
-org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
-org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled
-org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
-org.eclipse.jdt.core.compiler.problem.unusedExceptionParameter=ignore
-org.eclipse.jdt.core.compiler.problem.unusedImport=warning
-org.eclipse.jdt.core.compiler.problem.unusedLabel=warning
-org.eclipse.jdt.core.compiler.problem.unusedLocal=warning
-org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
-org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
-org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled
-org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
-org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
-org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning
-org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore
-org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning
-org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning
-org.eclipse.jdt.core.compiler.source=1.7
-org.eclipse.jdt.core.formatter.align_type_members_on_columns=false
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16
-org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_assignment=0
-org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16
-org.eclipse.jdt.core.formatter.alignment_for_compact_if=16
-org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80
-org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0
-org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16
-org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0
-org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16
-org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80
-org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16
-org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16
-org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16
-org.eclipse.jdt.core.formatter.blank_lines_after_imports=1
-org.eclipse.jdt.core.formatter.blank_lines_after_package=1
-org.eclipse.jdt.core.formatter.blank_lines_before_field=0
-org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0
-org.eclipse.jdt.core.formatter.blank_lines_before_imports=1
-org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1
-org.eclipse.jdt.core.formatter.blank_lines_before_method=1
-org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1
-org.eclipse.jdt.core.formatter.blank_lines_before_package=0
-org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1
-org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1
-org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line
-org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line
-org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false
-org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false
-org.eclipse.jdt.core.formatter.comment.format_block_comments=false
-org.eclipse.jdt.core.formatter.comment.format_header=false
-org.eclipse.jdt.core.formatter.comment.format_html=true
-org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false
-org.eclipse.jdt.core.formatter.comment.format_line_comments=false
-org.eclipse.jdt.core.formatter.comment.format_source_code=true
-org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true
-org.eclipse.jdt.core.formatter.comment.indent_root_tags=true
-org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert
-org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert
-org.eclipse.jdt.core.formatter.comment.line_length=80
-org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true
-org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true
-org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false
-org.eclipse.jdt.core.formatter.compact_else_if=true
-org.eclipse.jdt.core.formatter.continuation_indentation=2
-org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2
-org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off
-org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on
-org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false
-org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true
-org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true
-org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true
-org.eclipse.jdt.core.formatter.indent_empty_lines=false
-org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true
-org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true
-org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true
-org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false
-org.eclipse.jdt.core.formatter.indentation.size=4
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert
-org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert
-org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert
-org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert
-org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert
-org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert
-org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert
-org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert
-org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert
-org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert
-org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert
-org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert
-org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert
-org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert
-org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert
-org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert
-org.eclipse.jdt.core.formatter.join_lines_in_comments=true
-org.eclipse.jdt.core.formatter.join_wrapped_lines=true
-org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false
-org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false
-org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false
-org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false
-org.eclipse.jdt.core.formatter.lineSplit=999
-org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false
-org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false
-org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0
-org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1
-org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true
-org.eclipse.jdt.core.formatter.tabulation.char=space
-org.eclipse.jdt.core.formatter.tabulation.size=4
-org.eclipse.jdt.core.formatter.use_on_off_tags=false
-org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false
-org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true
-org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true
-org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/src/.settings/org.eclipse.jdt.ui.prefs
----------------------------------------------------------------------
diff --git a/src/.settings/org.eclipse.jdt.ui.prefs b/src/.settings/org.eclipse.jdt.ui.prefs
deleted file mode 100644
index d521bab..0000000
--- a/src/.settings/org.eclipse.jdt.ui.prefs
+++ /dev/null
@@ -1,7 +0,0 @@
-eclipse.preferences.version=1
-formatter_profile=_Space Indent & Long Lines
-formatter_settings_version=12
-org.eclipse.jdt.ui.ignorelowercasenames=true
-org.eclipse.jdt.ui.importorder=java;javax;org;com;
-org.eclipse.jdt.ui.ondemandthreshold=99
-org.eclipse.jdt.ui.staticondemandthreshold=99

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/src/main/config/assemblies/source-assembly.xml
----------------------------------------------------------------------
diff --git a/src/main/config/assemblies/source-assembly.xml b/src/main/config/assemblies/source-assembly.xml
deleted file mode 100644
index 23a215b..0000000
--- a/src/main/config/assemblies/source-assembly.xml
+++ /dev/null
@@ -1,117 +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.
--->
-<assembly>
-    <id>src</id>
-    <formats>
-        <format>zip</format>
-        <format>tar.gz</format>
-    </formats>
-    
-    <fileSets>
-        <!-- main project directory structure -->
-        <fileSet>
-            <directory>.</directory>
-            <outputDirectory>.</outputDirectory>
-            <useDefaultExcludes>true</useDefaultExcludes>
-            <excludes>
-                <!-- build output -->
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/).*${project.build.directory}.*]
-                </exclude>
-
-                <!-- NOTE: Most of the following excludes should not be required
-                  if the standard release process is followed. This is because the release
-                  plugin checks out project sources into a location like target/checkout, then
-                  runs the build from there. The result is a source-release archive that comes
-                  from a pretty clean directory structure. HOWEVER, if the release plugin is
-                  configured to run extra goals or generate a project website, it's definitely
-                  possible that some of these files will be present. So, it's safer to exclude
-                  them. -->
-
-                <!-- IDEs -->
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?maven-eclipse\.xml]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.project]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.classpath]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iws]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.idea(/.*)?]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?out(/.*)?]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.ipr]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iml]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.settings(/.*)?]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.externalToolBuilders(/.*)?]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.deployables(/.*)?]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.wtpmodules(/.*)?]
-                </exclude>
-
-
-                <!-- scm -->
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.gitignore(/.*)?]
-                </exclude>
-
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?docs/website(/.*)?]
-                </exclude>
-
-                <!-- release-plugin temp files -->
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.releaseBackup]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.next]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.tag]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]
-                </exclude>
-
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]
-                </exclude>
-
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?dist(/.*)?]
-                </exclude>
-
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?build(/.*)?]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?lib(/.*)?]
-                </exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?docs(/.*)?]
-                </exclude>
-            </excludes>
-        </fileSet>
-        <!-- LICENSE, NOTICE, DEPENDENCIES, git.properties, etc. calculated at build time -->
-        <fileSet>
-            <directory>${project.build.directory}/maven-shared-archive-resources/META-INF
-            </directory>
-            <outputDirectory>.</outputDirectory>
-        </fileSet>
-        <fileSet>
-            <directory>${project.build.directory}</directory>
-            <includes>
-                <include>git.properties</include>
-            </includes>
-            <outputDirectory>.</outputDirectory>
-        </fileSet>
-    </fileSets>
-</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/src/main/config/checkstyle/checkstyle-java-header.txt
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/checkstyle-java-header.txt b/src/main/config/checkstyle/checkstyle-java-header.txt
deleted file mode 100644
index c486fd3..0000000
--- a/src/main/config/checkstyle/checkstyle-java-header.txt
+++ /dev/null
@@ -1,17 +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.
- */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/src/main/config/checkstyle/checkstyle.xml
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/checkstyle.xml b/src/main/config/checkstyle/checkstyle.xml
deleted file mode 100644
index eab23dc..0000000
--- a/src/main/config/checkstyle/checkstyle.xml
+++ /dev/null
@@ -1,44 +0,0 @@
-<?xml version="1.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.
--->
-
-<!--
-  This version of checkstyle is based on the Apache Giraph checkstyle
-  configuration, which in turn is based on Hadoop and common-math
-  configurations.
-
-  The documentation for checkstyle is available at
-
-  http://checkstyle.sourceforge.net
--->
-
-<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.1//EN" "http://www.puppycrawl.com/dtds/configuration_1_1.dtd">
-
-<module name="Checker">
-    <property name="localeLanguage" value="en"/>
-    <property name="charset" value="UTF-8"/>
-
-
-    <!-- Checks for headers -->
-    <!-- See http://checkstyle.sf.net/config_header.html -->
-    <!-- Verify that EVERY source file has the appropriate license -->
-    <module name="Header">
-        <property name="headerFile" value="${checkstyle.header.file}"/>
-        <property name="fileExtensions" value="java"/>
-    </module>
-
-</module>

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/src/main/config/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/suppressions.xml b/src/main/config/checkstyle/suppressions.xml
deleted file mode 100644
index c6edc5f..0000000
--- a/src/main/config/checkstyle/suppressions.xml
+++ /dev/null
@@ -1,32 +0,0 @@
-<?xml version="1.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.
-  -->
-
-<!DOCTYPE suppressions PUBLIC
-        "-//Puppy Crawl//DTD Suppressions 1.1//EN"
-        "http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
-
-<suppressions>
-    <!-- Suppress javadoc, line length, visibility checks for test code -->
-    <suppress checks="JavadocStyleCheck" files="Test*.java"/>
-    <suppress checks="JavadocTypeCheck" files="Test*.java"/>
-    <suppress checks="LineLength" files="Test*.java"/>
-    <suppress checks="Indentation" files="Test*.java"/>
-    <suppress checks="VisibilityModifier" files="Test*.java"/>
-
-</suppressions>

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index 4fe7748..d5b36df 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -1,36 +1,21 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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.storage.hbase.steps;
 
 import java.io.IOException;


[05/50] [abbrv] kylin git commit: Revert "change to upper case"

Posted by li...@apache.org.
Revert "change to upper case"

This reverts commit aa30880578078369a5844e04a7d7ce736661e902.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: d1e979b4995597ae2ac3fbeb88ba1902d7296782
Parents: 3ae2549
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:50:04 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../kafka/DEFAULT.STREAMING_TABLE.json          | 21 --------------------
 .../streaming/DEFAULT.STREAMING_TABLE.json      |  6 ------
 2 files changed, 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d1e979b4/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
deleted file mode 100644
index 6a64cce..0000000
--- a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
+++ /dev/null
@@ -1,21 +0,0 @@
-{
- 
-  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
-  "name": "DEFAULT.STREAMING_TABLE",
-  "topic": "test_streaming_table_topic_xyz",
-  "timeout": 60000,
-  "bufferSize": 65536,
-  "parserName": "org.apache.kylin.source.kafka.TimedJsonStreamParser",
-  "last_modified": 0,
-  "clusters": [
-    {
-      "brokers": [
-        {
-          "id": 0,
-          "host": "sandbox",
-          "port": 6667
-        }
-      ]
-    }
-  ]
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1e979b4/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json b/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
deleted file mode 100644
index 85a477b..0000000
--- a/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
+++ /dev/null
@@ -1,6 +0,0 @@
-{
-  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
-  "name": "DEFAULT.STREAMING_TABLE",
-  "type": "kafka",
-  "last_modified": 0
-}


[39/50] [abbrv] kylin git commit: KYLIN-1999, for UT/IT, enable compress

Posted by li...@apache.org.
KYLIN-1999, for UT/IT, enable compress

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 61a3e7ef1b9998fb2414170b2978d62d84b8a6c1
Parents: a25131e
Author: Yiming Liu <li...@gmail.com>
Authored: Thu Sep 8 11:06:34 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 25 15:40:03 2016 +0800

----------------------------------------------------------------------
 examples/test_case_data/localmeta/kylin.properties       | 3 +++
 examples/test_case_data/localmeta/kylin_hive_conf.xml    | 4 +---
 examples/test_case_data/localmeta/kylin_job_conf.xml     | 2 --
 examples/test_case_data/sandbox/kylin.properties         | 2 +-
 examples/test_case_data/sandbox/kylin_hive_conf.xml      | 2 --
 examples/test_case_data/sandbox/kylin_job_conf.xml       | 2 --
 examples/test_case_data/sandbox/kylin_job_conf_inmem.xml | 2 --
 7 files changed, 5 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/61a3e7ef/examples/test_case_data/localmeta/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin.properties b/examples/test_case_data/localmeta/kylin.properties
index 50dd0b6..866a518 100644
--- a/examples/test_case_data/localmeta/kylin.properties
+++ b/examples/test_case_data/localmeta/kylin.properties
@@ -38,6 +38,9 @@ kylin.storage.url=hbase
 # Working folder in HDFS, make sure user has the right access to the hdfs directory
 kylin.hdfs.working.dir=/kylin
 
+# Compression codec for htable, valid value [none, snappy, lzo, gzip, lz4]
+kylin.hbase.default.compression.codec=snappy
+
 ### JOB ###
 
 # max job retry on error, default 0: no retry

http://git-wip-us.apache.org/repos/asf/kylin/blob/61a3e7ef/examples/test_case_data/localmeta/kylin_hive_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin_hive_conf.xml b/examples/test_case_data/localmeta/kylin_hive_conf.xml
index 61ef949..98e3121 100644
--- a/examples/test_case_data/localmeta/kylin_hive_conf.xml
+++ b/examples/test_case_data/localmeta/kylin_hive_conf.xml
@@ -40,7 +40,6 @@
         <description>enable map-side join</description>
     </property>
 
-    <!--
     <property>
         <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
@@ -51,8 +50,7 @@
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description></description>
     </property>
-    -->
-    
+
     <property>
         <name>hive.merge.mapfiles</name>
         <value>true</value>

http://git-wip-us.apache.org/repos/asf/kylin/blob/61a3e7ef/examples/test_case_data/localmeta/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin_job_conf.xml b/examples/test_case_data/localmeta/kylin_job_conf.xml
index 2f0628b..9d58241 100644
--- a/examples/test_case_data/localmeta/kylin_job_conf.xml
+++ b/examples/test_case_data/localmeta/kylin_job_conf.xml
@@ -27,7 +27,6 @@
     </property>
 
 
-    <!--
     <property>
         <name>mapred.compress.map.output</name>
         <value>true</value>
@@ -59,7 +58,6 @@
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>
-    -->
 
     <property>
         <name>mapreduce.job.max.split.locations</name>

http://git-wip-us.apache.org/repos/asf/kylin/blob/61a3e7ef/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index 1d1d9ba..8f3075e 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -28,7 +28,7 @@ kylin.owner=whoami@kylin.apache.org
 kylin.rest.servers=localhost:7070
 
 # Display timezone on UI,format like[GMT+N or GMT-N]
-kylin.rest.timezone=GMT-8
+kylin.rest.timezone=GMT+8
 
 ### SOURCE ###
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/61a3e7ef/examples/test_case_data/sandbox/kylin_hive_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_hive_conf.xml b/examples/test_case_data/sandbox/kylin_hive_conf.xml
index e75f191..4ee5f6b 100644
--- a/examples/test_case_data/sandbox/kylin_hive_conf.xml
+++ b/examples/test_case_data/sandbox/kylin_hive_conf.xml
@@ -40,7 +40,6 @@
         <description>enable map-side join</description>
     </property>
 
-    <!--
     <property>
         <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
@@ -51,7 +50,6 @@
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description></description>
     </property>
-    -->
 
     <property>
         <name>hive.merge.size.per.task</name>

http://git-wip-us.apache.org/repos/asf/kylin/blob/61a3e7ef/examples/test_case_data/sandbox/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_job_conf.xml b/examples/test_case_data/sandbox/kylin_job_conf.xml
index bd947af..e8f6399 100644
--- a/examples/test_case_data/sandbox/kylin_job_conf.xml
+++ b/examples/test_case_data/sandbox/kylin_job_conf.xml
@@ -31,7 +31,6 @@
         <value>2</value>
     </property>
 
-    <!--
     <property>
         <name>mapred.compress.map.output</name>
         <value>true</value>
@@ -63,7 +62,6 @@
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>
--->
 
     <property>
         <name>mapreduce.job.max.split.locations</name>

http://git-wip-us.apache.org/repos/asf/kylin/blob/61a3e7ef/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml b/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml
index 5f2cfba..8c77004 100644
--- a/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml
+++ b/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml
@@ -31,7 +31,6 @@
         <value>2</value>
     </property>
 
-    <!--
     <property>
         <name>mapred.compress.map.output</name>
         <value>true</value>
@@ -63,7 +62,6 @@
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>
--->
 
     <property>
         <name>mapreduce.job.max.split.locations</name>


[38/50] [abbrv] kylin git commit: KYLIN-1978: fix compatible issue on Ubuntu

Posted by li...@apache.org.
KYLIN-1978: fix compatible issue on Ubuntu

Signed-off-by: shaofengshi <sh...@apache.org>


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: a25131eca336c0698a75c85c64dc000c8b91cc74
Parents: e10f2b9
Author: Yiming Liu <li...@gmail.com>
Authored: Sun Sep 18 10:35:17 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Sep 25 15:38:21 2016 +0800

----------------------------------------------------------------------
 build/bin/get-properties.sh | 10 ++--------
 1 file changed, 2 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a25131ec/build/bin/get-properties.sh
----------------------------------------------------------------------
diff --git a/build/bin/get-properties.sh b/build/bin/get-properties.sh
index a53de6b..516e8c9 100755
--- a/build/bin/get-properties.sh
+++ b/build/bin/get-properties.sh
@@ -23,11 +23,5 @@ then
     exit -1
 fi
 
-IFS=$'\n'
-result=
-for i in `cat ${KYLIN_HOME}/conf/kylin.properties | grep -w "^$1" | grep -v '^#' | awk -F= '{ n = index($0,"="); print substr($0,n+1)}' | cut -c 1-`
-do
-   :
-   result=$i
-done
-echo $result
\ No newline at end of file
+result=`cat ${KYLIN_HOME}/conf/kylin.properties | grep -w "^$1" | grep -v '^#' | awk -F= '{ n = index($0,"="); print substr($0,n+1)}' | cut -c 1- |tail -1`
+echo $result


[30/50] [abbrv] kylin git commit: add .gitconfig to rat exclude list

Posted by li...@apache.org.
add .gitconfig to rat exclude list


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: f2732ec85f750441a1386115daafc6e2bb91647f
Parents: b6cdc0d
Author: shaofengshi <sh...@apache.org>
Authored: Fri Sep 23 23:56:02 2016 +0000
Committer: shaofengshi <sh...@apache.org>
Committed: Sat Sep 24 09:09:36 2016 +0800

----------------------------------------------------------------------
 pom.xml | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/f2732ec8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ecb2724..fbb27a4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1131,6 +1131,7 @@
                                 <exclude>**/LICENSE*</exclude>
                                 <!-- Local git repo -->
                                 <exclude>.git/**</exclude>
+                                <exclude>.gitconfig</exclude>
                                 <!-- IDE files -->
                                 <exclude>.idea/**</exclude>
                                 <exclude>**/*.iml</exclude>
@@ -1272,4 +1273,4 @@
             </build>
         </profile>
     </profiles>
-</project>
\ No newline at end of file
+</project>


[45/50] [abbrv] kylin git commit: KYLIN-2041: fix GET Hive Tables parameter definition

Posted by li...@apache.org.
KYLIN-2041: fix GET Hive Tables parameter definition

Signed-off-by: Yang Li <li...@apache.org>


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: abc6a744783731b9d1b33166a98c6fcdda5fddb8
Parents: 0def97c
Author: Yiming Liu <li...@gmail.com>
Authored: Fri Sep 23 21:59:59 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Mon Sep 26 19:58:21 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/rest/controller/TableController.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/abc6a744/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
----------------------------------------------------------------------
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 eefeba8..c4af5f4 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
@@ -90,7 +90,7 @@ public class TableController extends BasicController {
      */
     @RequestMapping(value = "", method = { RequestMethod.GET })
     @ResponseBody
-    public List<TableDesc> getHiveTables(@RequestParam(value = "ext", required = false) boolean withExt, @RequestParam(value = "project", required = false) String project) {
+    public List<TableDesc> getHiveTables(@RequestParam(value = "ext", required = false) boolean withExt, @RequestParam(value = "project", required = true) String project) {
         long start = System.currentTimeMillis();
         List<TableDesc> tables = null;
         try {


[44/50] [abbrv] kylin git commit: KYLIN-1995: Upgrade deprecated properties in testcases

Posted by li...@apache.org.
KYLIN-1995: Upgrade deprecated properties in testcases


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 0def97cce3e31db675be9aaedcfc0b8293d7f35a
Parents: 2993e7b
Author: Yiming Liu <li...@gmail.com>
Authored: Sun Sep 4 21:52:18 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Mon Sep 26 19:51:11 2016 +0800

----------------------------------------------------------------------
 examples/test_case_data/localmeta/kylin_job_conf.xml     | 10 +++++-----
 examples/test_case_data/sandbox/kylin_job_conf.xml       | 10 +++++-----
 examples/test_case_data/sandbox/kylin_job_conf_inmem.xml | 10 +++++-----
 examples/test_case_data/sandbox/mapred-site.xml          |  4 ++--
 4 files changed, 17 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0def97cc/examples/test_case_data/localmeta/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin_job_conf.xml b/examples/test_case_data/localmeta/kylin_job_conf.xml
index 9d58241..7755f0c 100644
--- a/examples/test_case_data/localmeta/kylin_job_conf.xml
+++ b/examples/test_case_data/localmeta/kylin_job_conf.xml
@@ -28,33 +28,33 @@
 
 
     <property>
-        <name>mapred.compress.map.output</name>
+        <name>mapreduce.map.output.compress</name>
         <value>true</value>
         <description>Compress map outputs</description>
     </property>
 
     <property>
-        <name>mapred.map.output.compression.codec</name>
+        <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for map outputs
         </description>
     </property>
 
     <property>
-        <name>mapred.output.compress</name>
+        <name>mapreduce.output.fileoutputformat.compress</name>
         <value>true</value>
         <description>Compress the output of a MapReduce job</description>
     </property>
 
     <property>
-        <name>mapred.output.compression.codec</name>
+        <name>mapreduce.output.fileoutputformat.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for job outputs
         </description>
     </property>
 
     <property>
-        <name>mapred.output.compression.type</name>
+        <name>mapreduce.output.fileoutputformat.compress.type</name>
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0def97cc/examples/test_case_data/sandbox/kylin_job_conf.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_job_conf.xml b/examples/test_case_data/sandbox/kylin_job_conf.xml
index e8f6399..27f4fb8 100644
--- a/examples/test_case_data/sandbox/kylin_job_conf.xml
+++ b/examples/test_case_data/sandbox/kylin_job_conf.xml
@@ -32,33 +32,33 @@
     </property>
 
     <property>
-        <name>mapred.compress.map.output</name>
+        <name>mapreduce.map.output.compress</name>
         <value>true</value>
         <description>Compress map outputs</description>
     </property>
 
     <property>
-        <name>mapred.map.output.compression.codec</name>
+        <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for map outputs
         </description>
     </property>
 
     <property>
-        <name>mapred.output.compress</name>
+        <name>mapreduce.output.fileoutputformat.compress</name>
         <value>true</value>
         <description>Compress the output of a MapReduce job</description>
     </property>
 
     <property>
-        <name>mapred.output.compression.codec</name>
+        <name>mapreduce.output.fileoutputformat.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for job outputs
         </description>
     </property>
 
     <property>
-        <name>mapred.output.compression.type</name>
+        <name>mapreduce.output.fileoutputformat.compress.type</name>
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0def97cc/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml b/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml
index 8c77004..42f1cc4 100644
--- a/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml
+++ b/examples/test_case_data/sandbox/kylin_job_conf_inmem.xml
@@ -32,33 +32,33 @@
     </property>
 
     <property>
-        <name>mapred.compress.map.output</name>
+        <name>mapreduce.map.output.compress</name>
         <value>true</value>
         <description>Compress map outputs</description>
     </property>
 
     <property>
-        <name>mapred.map.output.compression.codec</name>
+        <name>mapreduce.map.output.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for map outputs
         </description>
     </property>
 
     <property>
-        <name>mapred.output.compress</name>
+        <name>mapreduce.output.fileoutputformat.compress</name>
         <value>true</value>
         <description>Compress the output of a MapReduce job</description>
     </property>
 
     <property>
-        <name>mapred.output.compression.codec</name>
+        <name>mapreduce.output.fileoutputformat.compress.codec</name>
         <value>org.apache.hadoop.io.compress.SnappyCodec</value>
         <description>The compression codec to use for job outputs
         </description>
     </property>
 
     <property>
-        <name>mapred.output.compression.type</name>
+        <name>mapreduce.output.fileoutputformat.compress.type</name>
         <value>BLOCK</value>
         <description>The compression type to use for job outputs</description>
     </property>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0def97cc/examples/test_case_data/sandbox/mapred-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/mapred-site.xml b/examples/test_case_data/sandbox/mapred-site.xml
index 18f6feb..e90f594 100644
--- a/examples/test_case_data/sandbox/mapred-site.xml
+++ b/examples/test_case_data/sandbox/mapred-site.xml
@@ -27,12 +27,12 @@
     </property>
 
     <property>
-        <name>mapred.job.map.memory.mb</name>
+        <name>mapreduce.map.memory.mb</name>
         <value>512</value>
     </property>
 
     <property>
-        <name>mapred.job.reduce.memory.mb</name>
+        <name>mapreduce.reduce.memory.mb</name>
         <value>512</value>
     </property>
 


[35/50] [abbrv] kylin git commit: KYLIN-2047 Remove ineffective null check in MetadataManager

Posted by li...@apache.org.
KYLIN-2047 Remove ineffective null check in MetadataManager


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: afa350f0018fce4d34692fb0a1616829e11977ae
Parents: a346db5
Author: lidongsjtu <li...@apache.org>
Authored: Sat Sep 24 10:32:47 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Sat Sep 24 10:42:46 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/metadata/MetadataManager.java  | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/afa350f0/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java b/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
index 9be3faf..8d77131 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
@@ -263,7 +263,7 @@ public class MetadataManager {
         reloadAllSourceTableExd();
         reloadAllDataModel();
         reloadAllExternalFilter();
-        
+
         // touch lower level metadata before registering my listener
         Broadcaster.getInstance(config).registerListener(new SrcTableSyncListener(), "table");
         Broadcaster.getInstance(config).registerListener(new SrcTableExtSyncListener(), "table_ext");
@@ -283,7 +283,7 @@ public class MetadataManager {
                 srcTableMap.removeLocal(cacheKey);
             else
                 reloadSourceTable(cacheKey);
-            
+
             for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByTable(cacheKey)) {
                 broadcaster.notifyProjectSchemaUpdate(prj.getName());
             }
@@ -302,7 +302,7 @@ public class MetadataManager {
                 srcTableExdMap.removeLocal(cacheKey);
             else
                 reloadSourceTableExt(cacheKey);
-            
+
             for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByTable(cacheKey)) {
                 broadcaster.notifyProjectSchemaUpdate(prj.getName());
             }
@@ -328,7 +328,7 @@ public class MetadataManager {
                 dataModelDescMap.removeLocal(cacheKey);
             else
                 reloadDataModelDesc(cacheKey);
-            
+
             for (ProjectInstance prj : ProjectManager.getInstance(config).findProjectsByModel(cacheKey)) {
                 broadcaster.notifyProjectSchemaUpdate(prj.getName());
             }
@@ -549,8 +549,7 @@ public class MetadataManager {
     public DataModelDesc dropModel(DataModelDesc desc) throws IOException {
         logger.info("Dropping model '" + desc.getName() + "'");
         ResourceStore store = getStore();
-        if (desc != null)
-            store.deleteResource(desc.getResourcePath());
+        store.deleteResource(desc.getResourcePath());
         // delete model from project
         ProjectManager.getInstance(config).removeModelFromProjects(desc.getName());
         // clean model cache


[13/50] [abbrv] kylin git commit: KYLIN-2030 query on lookup table with no derived

Posted by li...@apache.org.
KYLIN-2030 query on lookup table with no derived


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: dbc02b75b09e07dbeee7bd417a29e5e9a01a0adf
Parents: 472e6d5
Author: Hongbin Ma <ma...@apache.org>
Authored: Tue Sep 20 18:45:36 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 21:17:12 2016 +0800

----------------------------------------------------------------------
 .../kylin/cube/CubeCapabilityChecker.java       | 24 +++++++++++++++++---
 1 file changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/dbc02b75/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index caef529..1eada16 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -25,14 +25,18 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
+import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
+import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.SQLDigest;
@@ -73,7 +77,21 @@ public class CubeCapabilityChecker {
 
         // try dimension-as-measure
         if (!unmatchedAggregations.isEmpty()) {
-            tryDimensionAsMeasures(unmatchedAggregations, digest, cube, result);
+            if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+                tryDimensionAsMeasures(unmatchedAggregations, digest, cube, result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
+            } else {
+                //deal with query on lookup table, like https://issues.apache.org/jira/browse/KYLIN-2030
+                if (cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
+                    TableDesc tableDesc = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
+                    Set<TblColRef> dimCols = Sets.newHashSet();
+                    for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+                        dimCols.add(columnDesc.getRef());
+                    }
+                    tryDimensionAsMeasures(unmatchedAggregations, digest, cube, result, dimCols);
+                } else {
+                    logger.info("Skip tryDimensionAsMeasures because current cube {} does not touch lookup table {} at all", cube.getName(), digest.factTable);
+                }
+            }
         }
 
         if (!unmatchedDimensions.isEmpty()) {
@@ -129,7 +147,7 @@ public class CubeCapabilityChecker {
         return result;
     }
 
-    private static void tryDimensionAsMeasures(Collection<FunctionDesc> unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult result) {
+    private static void tryDimensionAsMeasures(Collection<FunctionDesc> unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult result, Set<TblColRef> dimCols) {
         CubeDesc cubeDesc = cube.getDescriptor();
         Collection<FunctionDesc> cubeFuncs = cubeDesc.listAllFunctions();
 
@@ -154,7 +172,7 @@ public class CubeCapabilityChecker {
                 continue;
             }
             List<TblColRef> neededCols = parameterDesc.getColRefs();
-            if (neededCols.size() > 0 && cubeDesc.listDimensionColumnsIncludingDerived().containsAll(neededCols) && FunctionDesc.BUILT_IN_AGGREGATIONS.contains(functionDesc.getExpression())) {
+            if (neededCols.size() > 0 && dimCols.containsAll(neededCols) && FunctionDesc.BUILT_IN_AGGREGATIONS.contains(functionDesc.getExpression())) {
                 result.influences.add(new CapabilityResult.DimensionAsMeasure(functionDesc));
                 it.remove();
                 continue;


[07/50] [abbrv] kylin git commit: Revert "rename the streaming_table.json"

Posted by li...@apache.org.
Revert "rename the streaming_table.json"

This reverts commit ffdc5d21d5f46daaf25a0dfed9b3afeef78fc62c.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: bec25b4638835301a102141e56e12cd38d09139f
Parents: d1e979b
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:50:13 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../kafka/DEFAULT.STREAMING_TABLE.json          | 21 ++++++++++++++++++++
 .../streaming/DEFAULT.STREAMING_TABLE.json      |  6 ++++++
 2 files changed, 27 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/bec25b46/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
new file mode 100644
index 0000000..6a64cce
--- /dev/null
+++ b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
@@ -0,0 +1,21 @@
+{
+ 
+  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
+  "name": "DEFAULT.STREAMING_TABLE",
+  "topic": "test_streaming_table_topic_xyz",
+  "timeout": 60000,
+  "bufferSize": 65536,
+  "parserName": "org.apache.kylin.source.kafka.TimedJsonStreamParser",
+  "last_modified": 0,
+  "clusters": [
+    {
+      "brokers": [
+        {
+          "id": 0,
+          "host": "sandbox",
+          "port": 6667
+        }
+      ]
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/bec25b46/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json b/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
new file mode 100644
index 0000000..85a477b
--- /dev/null
+++ b/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
@@ -0,0 +1,6 @@
+{
+  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
+  "name": "DEFAULT.STREAMING_TABLE",
+  "type": "kafka",
+  "last_modified": 0
+}


[03/50] [abbrv] kylin git commit: Revert "KYLIN-1818 change kafka dependency to provided"

Posted by li...@apache.org.
Revert "KYLIN-1818 change kafka dependency to provided"

This reverts commit 4e060e7bc89f757a138dfa5beba2a43209e91371.


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: dee8f2d073d225a3b8189c66170fd310c1a8d221
Parents: 355e58b
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:50:37 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 build/bin/find-kafka-dependency.sh                      | 12 ++++++------
 build/bin/kylin.sh                                      |  2 --
 .../kylin/engine/mr/common/AbstractHadoopJob.java       |  4 ++--
 3 files changed, 8 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/dee8f2d0/build/bin/find-kafka-dependency.sh
----------------------------------------------------------------------
diff --git a/build/bin/find-kafka-dependency.sh b/build/bin/find-kafka-dependency.sh
index 7349360..c6b9c24 100644
--- a/build/bin/find-kafka-dependency.sh
+++ b/build/bin/find-kafka-dependency.sh
@@ -32,20 +32,20 @@ then
 fi
 
 # works for kafka 9+
-kafka_dependency=`find -L $kafka_home -name 'kafka-clients-[a-z0-9A-Z\.-]*.jar' ! -name '*doc*' ! -name '*test*' ! -name '*sources*' ''-printf '%p:' | sed 's/:$//'`
-if [ -z "$kafka_dependency" ]
+kafka_client=`find -L "$(dirname $kafka_home)" -name 'kafka-clients-[a-z0-9A-Z\.-]*.jar' ! -name '*doc*' ! -name '*test*' ! -name '*sources*' ''-printf '%p:' | sed 's/:$//'`
+if [ -z "$kafka_client" ]
 then
 # works for kafka 8
-    kafka_dependency=`find -L $kafka_home -name 'kafka_[a-z0-9A-Z\.-]*.jar' ! -name '*doc*' ! -name '*test*' ! -name '*sources*' ''-printf '%p:' | sed 's/:$//'`
-    if [ -z "$kafka_dependency" ]
+    kafka_broker=`find -L "$(dirname $kafka_home)" -name 'kafka_[a-z0-9A-Z\.-]*.jar' ! -name '*doc*' ! -name '*test*' ! -name '*sources*' ''-printf '%p:' | sed 's/:$//'`
+    if [ -z "$kafka_broker" ]
     then
         echo "kafka client lib not found"
         exit 1
     else
-        echo "kafka dependency: $kafka_dependency"
+        echo "kafka dependency: $kafka_broker"
         export kafka_dependency
     fi
 else
-    echo "kafka dependency: $kafka_dependency"
+    echo "kafka dependency: $kafka_client"
     export kafka_dependency
 fi

http://git-wip-us.apache.org/repos/asf/kylin/blob/dee8f2d0/build/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/build/bin/kylin.sh b/build/bin/kylin.sh
index 7a9d2a1..201b5b6 100644
--- a/build/bin/kylin.sh
+++ b/build/bin/kylin.sh
@@ -31,7 +31,6 @@ function retrieveDependency() {
     #retrive $hive_dependency and $hbase_dependency
     source ${dir}/find-hive-dependency.sh
     source ${dir}/find-hbase-dependency.sh
-    source ${dir}/find-kafka-dependency.sh
 
     #retrive $KYLIN_EXTRA_START_OPTS
     if [ -f "${dir}/setenv.sh" ]
@@ -107,7 +106,6 @@ then
     -Djava.io.tmpdir=${tomcat_root}/temp  \
     -Dkylin.hive.dependency=${hive_dependency} \
     -Dkylin.hbase.dependency=${hbase_dependency} \
-    -Dkylin.kafka.dependency=${kafka_dependency} \
     -Dkylin.rest.address=${kylin_rest_address} \
     -Dspring.profiles.active=${spring_profile} \
     org.apache.hadoop.util.RunJar ${tomcat_root}/bin/bootstrap.jar  org.apache.catalina.startup.Bootstrap start >> ${KYLIN_HOME}/logs/kylin.out 2>&1 & echo $! > ${KYLIN_HOME}/pid &

http://git-wip-us.apache.org/repos/asf/kylin/blob/dee8f2d0/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index a138eec..af2ed9f 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -226,11 +226,11 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
             }
         }
 
-        // for kafka dependencies
+        // for hive dependencies
         if (kylinKafkaDependency != null) {
             kylinKafkaDependency = kylinKafkaDependency.replace(":", ",");
 
-            logger.info("Kafka Dependencies Before Filtered: " + kylinKafkaDependency);
+            logger.info("Kafka Dependencies Before Filtered: " + kylinHiveDependency);
 
             if (kylinDependency.length() > 0)
                 kylinDependency.append(",");


[19/50] [abbrv] kylin git commit: kylin advanced setting edit

Posted by li...@apache.org.
kylin advanced setting edit

Signed-off-by: Jason <ji...@163.com>


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 69cb49dbad6b27e9ca7662fa8dfaf29ad90cc98a
Parents: 45cc681
Author: luguosheng <55...@qq.com>
Authored: Wed Sep 21 12:21:55 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Wed Sep 21 14:27:27 2016 +0800

----------------------------------------------------------------------
 webapp/app/partials/cubeDesigner/advanced_settings.html | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/69cb49db/webapp/app/partials/cubeDesigner/advanced_settings.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/advanced_settings.html b/webapp/app/partials/cubeDesigner/advanced_settings.html
index b11fc8e..e8cbf9e 100755
--- a/webapp/app/partials/cubeDesigner/advanced_settings.html
+++ b/webapp/app/partials/cubeDesigner/advanced_settings.html
@@ -45,7 +45,7 @@
               <td class="col-xs-11">
                 <table class="table">
                   <tr class="row">
-                    <td class="col-xs-3">
+                    <td class="col-xs-3" style="min-width:130px;">
                       Includes
                     </td>
                     <td class="col-xs-8">
@@ -189,7 +189,7 @@
                   ng-show="state.mode=='edit'">New Aggregation Group<i class="fa fa-plus"></i>
           </button>
         </div>
-        <div class="form-group large-popover" style="overflow:auto">
+        <div class="form-group large-popover" >
           <h3 style="margin-left:42px">Rowkeys  <i kylinpopover placement="right" title="Rowkey" template="rowkeyTip.html" class="fa fa-info-circle"></i></h3>
 
           <div style="margin-left:42px">


[34/50] [abbrv] kylin git commit: KYLIN-2045 Fix argument order in JobInstanceExtractor#executeExtract()

Posted by li...@apache.org.
KYLIN-2045 Fix argument order in JobInstanceExtractor#executeExtract()


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 818188d484f2a848c84d3b1f79865e04e74dcd22
Parents: 5658283
Author: lidongsjtu <li...@apache.org>
Authored: Sat Sep 24 10:27:25 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Sat Sep 24 10:42:46 2016 +0800

----------------------------------------------------------------------
 tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/818188d4/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java b/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
index ef77c6a..1023a8b 100644
--- a/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
+++ b/tool/src/main/java/org/apache/kylin/tool/JobInstanceExtractor.java
@@ -85,7 +85,7 @@ public class JobInstanceExtractor extends AbstractInfoExtractor {
 
         long endTime = System.currentTimeMillis();
         long startTime = endTime - period * 24 * 3600 * 1000; // time in Millis
-        List<JobInstance> jobInstances = listJobInstances(cube, project, startTime, endTime);
+        List<JobInstance> jobInstances = listJobInstances(project, cube, startTime, endTime);
         logger.info("There are {} jobInstances to extract.", jobInstances.size());
 
         ObjectMapper mapper = new ObjectMapper();


[21/50] [abbrv] kylin git commit: minor changes

Posted by li...@apache.org.
minor changes


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 2359a75e3ca9d1a9c5f06331ab02edce1385b66d
Parents: 32a9c67
Author: Hongbin Ma <ma...@apache.org>
Authored: Thu Sep 22 10:45:08 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Thu Sep 22 10:45:15 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/metadata/datatype/BooleanSerializer.java | 2 +-
 .../kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java      | 1 +
 .../java/org/apache/kylin/rest/controller/QueryController.java     | 2 +-
 3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/2359a75e/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
index e843247..acb6de1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
@@ -25,7 +25,7 @@ import org.apache.commons.lang.BooleanUtils;
 
 public class BooleanSerializer extends DataTypeSerializer<LongMutable> {
 
-    final String[] TRUE_VALUE_SET = { "true", "t", "on", "yes" };
+    public final static String[] TRUE_VALUE_SET = { "true", "t", "on", "yes" };
 
     // be thread-safe and avoid repeated obj creation
     private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();

http://git-wip-us.apache.org/repos/asf/kylin/blob/2359a75e/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
----------------------------------------------------------------------
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
index 0e40150..2f2691e 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
@@ -94,6 +94,7 @@ public class SortedIteratorMergerWithLimit<E extends Cloneable> extends SortedIt
                 PeekingImpl<E> first = heap.poll();
                 E current = first.next();
                 try {
+                    //clone is protected on Object, have to use reflection to call the overwritten clone method in subclasses
                     current = (E) current.getClass().getMethod("clone").invoke(current);
                 } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
                     throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/kylin/blob/2359a75e/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
index 5cf6492..62cb0a7 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
@@ -205,7 +205,7 @@ public class QueryController extends BasicController {
                 checkQueryAuth(sqlResponse);
 
             } catch (Throwable e) { // calcite may throw AssertError
-                //logger.error("Exception when execute sql", e);
+                logger.error("Exception when execute sql", e);
                 String errMsg = QueryUtil.makeErrorMsgUserFriendly(e);
 
                 sqlResponse = new SQLResponse(null, null, 0, true, errMsg);


[18/50] [abbrv] kylin git commit: KYLIN-2010 UI filter encoding options for different column type

Posted by li...@apache.org.
KYLIN-2010 UI filter encoding options for different column type


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 45cc681326bbd701fd8740c6425354f362f1d87e
Parents: c1fc723
Author: Jason <ji...@163.com>
Authored: Wed Sep 21 14:20:48 2016 +0800
Committer: Jason <ji...@163.com>
Committed: Wed Sep 21 14:20:48 2016 +0800

----------------------------------------------------------------------
 webapp/app/js/controllers/cubeEdit.js           | 41 +++++++++++++++++++-
 webapp/app/js/model/cubeConfig.js               |  6 ++-
 webapp/app/js/model/tableModel.js               |  2 +
 .../cubeDesigner/advanced_settings.html         |  3 +-
 4 files changed, 49 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/45cc6813/webapp/app/js/controllers/cubeEdit.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeEdit.js b/webapp/app/js/controllers/cubeEdit.js
index e2d0ab5..e9bd93e 100755
--- a/webapp/app/js/controllers/cubeEdit.js
+++ b/webapp/app/js/controllers/cubeEdit.js
@@ -40,11 +40,50 @@ KylinApp.controller('CubeEditCtrl', function ($scope, $q, $routeParams, $locatio
     supportedEncoding:[]
   }
   CubeService.getValidEncodings({}, function (encodings) {
-    $scope.store.supportedEncoding = encodings;
+    for(var i =0;i<encodings.length;i++){
+      var value = encodings[i];
+      var name = value;
+      if(value=="int"){
+        name = "int (deprecated)";
+      }
+      $scope.store.supportedEncoding.push({
+        "name":name,
+        "value":value
+      });
+    }
   },function(e){
     $scope.store.supportedEncoding = $scope.cubeConfig.encodings;
   })
 
+  $scope.getEncodings =function (name){
+    var type = TableModel.columnNameTypeMap[name];
+    var encodings = $scope.store.supportedEncoding;
+    if(!type){
+      return encodings;
+    }
+    var filterEncodings = [];
+    for(var i = 0;i<encodings.length;i++){
+      var encodingValue = encodings[i].value;
+      if(encodingValue == "fixed_length_hex" ){
+        if(type.indexOf("varchar")!==-1){
+          filterEncodings.push(encodings[i]);
+        }
+      }else if(encodingValue == "date"){
+        if(type=="date"){
+          filterEncodings.push(encodings[i]);
+        }
+      }else if(encodingValue == "time"){
+        if(type=="time"||type=="datetime"||type=="timestamp"){
+          filterEncodings.push(encodings[i]);
+        }
+      }else{
+        filterEncodings.push(encodings[i]);
+      }
+    }
+
+    return filterEncodings;
+  }
+
   $scope.getColumnsByTable = function (tableName) {
     var temp = [];
     angular.forEach(TableModel.selectProjectTables, function (table) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/45cc6813/webapp/app/js/model/cubeConfig.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeConfig.js b/webapp/app/js/model/cubeConfig.js
index 2876f40..a77ab4f 100644
--- a/webapp/app/js/model/cubeConfig.js
+++ b/webapp/app/js/model/cubeConfig.js
@@ -66,7 +66,11 @@ KylinApp.constant('cubeConfig', {
     topN:{name: 'Top 100', value: "topn(100)"}
   },
     dictionaries: ["true", "false"],
-    encodings:["dict","fixed_length","int"],
+    encodings:[
+      {name:"dict",value:"value"},
+      {name:"fixed_length",value:"fixed_length"},
+      {name:"int (deprecated)",value:"int"}
+    ],
     intEncodingOptions: [1,2,3,4,5,6,7,8],
 //    cubes config
   theaditems: [

http://git-wip-us.apache.org/repos/asf/kylin/blob/45cc6813/webapp/app/js/model/tableModel.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/tableModel.js b/webapp/app/js/model/tableModel.js
index 788fbae..87c9b9d 100755
--- a/webapp/app/js/model/tableModel.js
+++ b/webapp/app/js/model/tableModel.js
@@ -25,6 +25,7 @@ KylinApp.service('TableModel', function(ProjectModel,$q,TableService,$log) {
     var loading = false;
    //for tables in cubeDesigner
     this.selectProjectTables = [];
+    this.columnNameTypeMap = {};
 
     this.initTables = function(){
         this.selectProjectTables = [];
@@ -81,6 +82,7 @@ KylinApp.service('TableModel', function(ProjectModel,$q,TableService,$log) {
                         column.cardinality = null;
                     }
                     column.id = parseInt(column.id);
+                  _this.columnNameTypeMap[column.name] = column.datatype;
                 });
                 tableMap[table.database].push(table);
             });

http://git-wip-us.apache.org/repos/asf/kylin/blob/45cc6813/webapp/app/partials/cubeDesigner/advanced_settings.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/advanced_settings.html b/webapp/app/partials/cubeDesigner/advanced_settings.html
index 29d55a6..b11fc8e 100755
--- a/webapp/app/partials/cubeDesigner/advanced_settings.html
+++ b/webapp/app/partials/cubeDesigner/advanced_settings.html
@@ -229,11 +229,12 @@
 
                       <span ng-if="state.mode=='view'">{{rowkey_column.column}}</span>
                     </td>
+
                     <td>
                       <select ng-if="state.mode=='edit'" style="width:180px;"
                               chosen ng-model="rowkey_column.encoding"
                               ng-change="refreshRowKey(convertedRowkeys,$index,rowkey_column);"
-                              ng-options="dt as dt for dt in store.supportedEncoding">
+                              ng-options="dt.value as dt.name for dt in getEncodings(rowkey_column.column)">
                         <option value=""></option>
                       </select>
                       <span ng-if="state.mode=='view'">{{rowkey_column.encoding}}</span>


[25/50] [abbrv] kylin git commit: KYLIN-1828 StorageCleanupJob

Posted by li...@apache.org.
KYLIN-1828 StorageCleanupJob

Signed-off-by: Li Yang <li...@apache.org>


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 858fad676bb42366c26f4033fcc208675cb4bf72
Parents: fa5c9cb
Author: Cheng Wang <ch...@kyligence.io>
Authored: Fri Sep 23 17:32:24 2016 +0800
Committer: Li Yang <li...@apache.org>
Committed: Fri Sep 23 17:44:23 2016 +0800

----------------------------------------------------------------------
 .../storage/hbase/util/StorageCleanupJob.java   | 57 ++++++++++++++++----
 1 file changed, 48 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/858fad67/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index 4bd2c53..dffce36 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -29,6 +29,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.regex.Pattern;
 
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
@@ -51,6 +52,7 @@ import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
 import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -61,11 +63,13 @@ public class StorageCleanupJob extends AbstractApplication {
 
     @SuppressWarnings("static-access")
     protected static final Option OPTION_DELETE = OptionBuilder.withArgName("delete").hasArg().isRequired(false).withDescription("Delete the unused storage").create("delete");
+    protected static final Option OPTION_FORCE = OptionBuilder.withArgName("force").hasArg().isRequired(false).withDescription("Warning: will delete any intermediate hive tables").create("force");
 
     protected static final Logger logger = LoggerFactory.getLogger(StorageCleanupJob.class);
     public static final int TIME_THRESHOLD_DELETE_HTABLE = 10; // Unit minute
 
     protected boolean delete = false;
+    protected boolean force = false;
     protected static ExecutableManager executableManager = ExecutableManager.getInstance(KylinConfig.getInstanceFromEnv());
 
     private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
@@ -132,6 +136,7 @@ public class StorageCleanupJob extends AbstractApplication {
     protected Options getOptions() {
         Options options = new Options();
         options.addOption(OPTION_DELETE);
+        options.addOption(OPTION_FORCE);
         return options;
     }
 
@@ -139,7 +144,9 @@ public class StorageCleanupJob extends AbstractApplication {
     protected void execute(OptionsHelper optionsHelper) throws Exception {
         logger.info("options: '" + optionsHelper.getOptionsAsString() + "'");
         logger.info("delete option value: '" + optionsHelper.getOptionValue(OPTION_DELETE) + "'");
+        logger.info("force option value: '" + optionsHelper.getOptionValue(OPTION_FORCE) + "'");
         delete = Boolean.parseBoolean(optionsHelper.getOptionValue(OPTION_DELETE));
+        force = Boolean.parseBoolean(optionsHelper.getOptionValue(OPTION_FORCE));
 
         Configuration conf = HBaseConfiguration.create();
 
@@ -183,6 +190,7 @@ public class StorageCleanupJob extends AbstractApplication {
         // GlobFilter filter = new
         // GlobFilter(KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory()
         // + "/kylin-.*");
+        // TODO: when first use, /kylin/kylin_metadata does not exist.
         FileStatus[] fStatus = fs.listStatus(new Path(KylinConfig.getInstanceFromEnv().getHdfsWorkingDirectory()));
         for (FileStatus status : fStatus) {
             String path = status.getPath().getName();
@@ -242,6 +250,8 @@ public class StorageCleanupJob extends AbstractApplication {
         final KylinConfig config = KylinConfig.getInstanceFromEnv();
         final CliCommandExecutor cmdExec = config.getCliCommandExecutor();
         final int uuidLength = 36;
+        final String preFix = "kylin_intermediate_";
+        final String uuidPattern = "[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}";
 
         final String useDatabaseHql = "USE " + config.getHiveDatabaseForIntermediateTable() + ";";
         final HiveCmdBuilder hiveCmdBuilder = new HiveCmdBuilder();
@@ -260,7 +270,6 @@ public class StorageCleanupJob extends AbstractApplication {
         for (String jobId : allJobs) {
             // only remove FINISHED and DISCARDED job intermediate table
             final ExecutableState state = executableManager.getOutput(jobId).getState();
-
             if (!state.isFinalState()) {
                 workingJobList.add(jobId);
                 logger.info("Skip intermediate hive table with job id " + jobId + " with job status " + state);
@@ -268,18 +277,35 @@ public class StorageCleanupJob extends AbstractApplication {
         }
 
         while ((line = reader.readLine()) != null) {
-            if (line.startsWith("kylin_intermediate_")) {
-                boolean isNeedDel = false;
+            if (!line.startsWith(preFix))
+                continue;
+
+            if (force == true) {
+                logger.warn("!!!!!!!!!!!!!!!Warning: will delete all intermediate hive tables!!!!!!!!!!!!!!!!!!!!!!");
+                allHiveTablesNeedToBeDeleted.add(line);
+                continue;
+            }
+
+            boolean isNeedDel = true;
+
+            if (line.length() > preFix.length() + uuidLength) {
                 String uuid = line.substring(line.length() - uuidLength, line.length());
                 uuid = uuid.replace("_", "-");
-                //Check whether it's a hive table in use
-                if (allJobs.contains(uuid) && !workingJobList.contains(uuid)) {
-                    isNeedDel = true;
+                final Pattern UUId_PATTERN = Pattern.compile(uuidPattern);
+                if (UUId_PATTERN.matcher(uuid).matches()) {
+                    //Check whether it's a hive table in use
+                    if (isTableInUse(uuid, workingJobList)) {
+                        isNeedDel = false;
+                    }
+                } else {
+                    isNeedDel = false;
                 }
+            } else {
+                isNeedDel = false;
+            }
 
-                if (isNeedDel) {
-                    allHiveTablesNeedToBeDeleted.add(line);
-                }
+            if (isNeedDel) {
+                allHiveTablesNeedToBeDeleted.add(line);
             }
         }
 
@@ -308,6 +334,19 @@ public class StorageCleanupJob extends AbstractApplication {
             reader.close();
     }
 
+    private boolean isTableInUse(String segUuid, List<String> workingJobList) {
+        for (String jobId : workingJobList) {
+            AbstractExecutable abstractExecutable = executableManager.getJob(jobId);
+            String segmentId = abstractExecutable.getParam("segmentId");
+
+            if (null == segmentId)
+                continue;
+
+            return segUuid.equals(segmentId);
+        }
+        return false;
+    }
+
     public static void main(String[] args) throws Exception {
         StorageCleanupJob cli = new StorageCleanupJob();
         cli.execute(args);


[22/50] [abbrv] kylin git commit: KYLIN-1993 use sed instead of envsubst

Posted by li...@apache.org.
KYLIN-1993 use sed instead of envsubst


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: c2ff8c5135e72cafeae09f20e654d247f9335d5c
Parents: 2359a75
Author: lidongsjtu <li...@apache.org>
Authored: Thu Sep 22 11:01:39 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Thu Sep 22 12:54:03 2016 +0800

----------------------------------------------------------------------
 build/bin/sample.sh                                            | 6 +++---
 .../sample_cube/template/cube_desc/kylin_sales_cube_desc.json  | 4 ++--
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/c2ff8c51/build/bin/sample.sh
----------------------------------------------------------------------
diff --git a/build/bin/sample.sh b/build/bin/sample.sh
index c915fb4..92caace 100644
--- a/build/bin/sample.sh
+++ b/build/bin/sample.sh
@@ -48,12 +48,12 @@ if [ -z "$default_engine_type" ]; then
     default_engine_type=2
     default_storage_type=2
 fi
-export default_engine_type
-export default_storage_type
 
 mkdir -p ${KYLIN_HOME}/sample_cube/metadata
 cp -rf ${KYLIN_HOME}/sample_cube/template/* ${KYLIN_HOME}/sample_cube/metadata
-envsubst < ${KYLIN_HOME}/sample_cube/template/cube_desc/kylin_sales_cube_desc.json > ${KYLIN_HOME}/sample_cube/metadata/cube_desc/kylin_sales_cube_desc.json
+
+sed -i "s/%default_storage_type%/${default_storage_type}/g" ${KYLIN_HOME}/sample_cube/metadata/cube_desc/kylin_sales_cube_desc.json
+sed -i "s/%default_engine_type%/${default_engine_type}/g" ${KYLIN_HOME}/sample_cube/metadata/cube_desc/kylin_sales_cube_desc.json
 
 cd ${KYLIN_HOME}
 hbase org.apache.hadoop.util.RunJar ${job_jar} org.apache.kylin.common.persistence.ResourceTool upload ${KYLIN_HOME}/sample_cube/metadata  || { exit 1; }

http://git-wip-us.apache.org/repos/asf/kylin/blob/c2ff8c51/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json
----------------------------------------------------------------------
diff --git a/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json b/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json
index 0953be6..70b1294 100644
--- a/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json
+++ b/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json
@@ -176,8 +176,8 @@
   "status_need_notify" : [ ],
   "auto_merge_time_ranges" : null,
   "retention_range" : 0,
-  "engine_type" : ${default_engine_type},
-  "storage_type" : ${default_storage_type},
+  "engine_type" : %default_engine_type%,
+  "storage_type" : %default_storage_type%,
   "override_kylin_properties" : { },
   "partition_date_start" : 1325376000000
 }
\ No newline at end of file


[49/50] [abbrv] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
index 5b2441c..2f7e164 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
@@ -24,11 +24,11 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableContext;
@@ -69,19 +69,20 @@ public class MergeGCStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin admin = null;
+            Admin admin = null;
             try {
-                admin = new HBaseAdmin(conf);
+                Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+                admin = conn.getAdmin();
+
                 for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                    if (admin.tableExists(TableName.valueOf(table))) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf((table)));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
+                            if (admin.isTableEnabled(TableName.valueOf(table))) {
+                                admin.disableTable(TableName.valueOf(table));
                             }
-                            admin.deleteTable(table);
+                            admin.deleteTable(TableName.valueOf(table));
                             logger.debug("Dropped htable: " + table);
                             output.append("HBase table " + table + " is dropped. \n");
                         } else {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
index a150607..56f867a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
@@ -21,9 +21,11 @@ package org.apache.kylin.storage.hbase.util;
 import java.io.IOException;
 
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -38,8 +40,8 @@ public class CleanHtableCLI extends AbstractApplication {
     protected static final Logger logger = LoggerFactory.getLogger(CleanHtableCLI.class);
 
     private void clean() throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
 
         for (HTableDescriptor descriptor : hbaseAdmin.listTables()) {
             String name = descriptor.getNameAsString().toLowerCase();
@@ -50,7 +52,7 @@ public class CleanHtableCLI extends AbstractApplication {
                 System.out.println();
 
                 descriptor.setValue(IRealizationConstants.HTableOwner, "DL-eBay-Kylin@ebay.com");
-                hbaseAdmin.modifyTable(descriptor.getNameAsString(), descriptor);
+                hbaseAdmin.modifyTable(TableName.valueOf(descriptor.getNameAsString()), descriptor);
             }
         }
         hbaseAdmin.close();

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index 52aa7ea..b78e3d7 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -33,12 +33,13 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RawResource;
@@ -86,7 +87,7 @@ public class CubeMigrationCLI {
     private static ResourceStore srcStore;
     private static ResourceStore dstStore;
     private static FileSystem hdfsFS;
-    private static HBaseAdmin hbaseAdmin;
+    private static Admin hbaseAdmin;
 
     public static final String ACL_INFO_FAMILY = "i";
     private static final String ACL_TABLE_NAME = "_acl";
@@ -130,8 +131,8 @@ public class CubeMigrationCLI {
 
         checkAndGetHbaseUrl();
 
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(srcConfig.getStorageUrl());
+        hbaseAdmin = conn.getAdmin();
 
         hdfsFS = FileSystem.get(new Configuration());
 
@@ -229,6 +230,7 @@ public class CubeMigrationCLI {
             operations.add(new Opt(OptType.COPY_DICT_OR_SNAPSHOT, new Object[] { item, cube.getName() }));
         }
     }
+
     private static void addCubeAndModelIntoProject(CubeInstance srcCube, String cubeName, String projectName) throws IOException {
         String projectResPath = ProjectInstance.concatResourcePath(projectName);
         if (!dstStore.exists(projectResPath))
@@ -322,8 +324,8 @@ public class CubeMigrationCLI {
 
         switch (opt.type) {
         case CHANGE_HTABLE_HOST: {
-            String tableName = (String) opt.params[0];
-            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            TableName tableName = TableName.valueOf((String) opt.params[0]);
+            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(tableName);
             hbaseAdmin.disableTable(tableName);
             desc.setValue(IRealizationConstants.HTableTag, dstConfig.getMetadataUrlPrefix());
             hbaseAdmin.modifyTable(tableName, desc);
@@ -445,11 +447,11 @@ public class CubeMigrationCLI {
             Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
             ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
             String projUUID = project.getUuid();
-            HTableInterface srcAclHtable = null;
-            HTableInterface destAclHtable = null;
+            Table srcAclHtable = null;
+            Table destAclHtable = null;
             try {
-                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(TableName.valueOf(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
 
                 // cube acl
                 Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
@@ -469,7 +471,6 @@ public class CubeMigrationCLI {
                         destAclHtable.put(put);
                     }
                 }
-                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(srcAclHtable);
                 IOUtils.closeQuietly(destAclHtable);
@@ -500,8 +501,8 @@ public class CubeMigrationCLI {
 
         switch (opt.type) {
         case CHANGE_HTABLE_HOST: {
-            String tableName = (String) opt.params[0];
-            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            TableName tableName = TableName.valueOf((String) opt.params[0]);
+            HTableDescriptor desc = hbaseAdmin.getTableDescriptor(tableName);
             hbaseAdmin.disableTable(tableName);
             desc.setValue(IRealizationConstants.HTableTag, srcConfig.getMetadataUrlPrefix());
             hbaseAdmin.modifyTable(tableName, desc);
@@ -535,13 +536,12 @@ public class CubeMigrationCLI {
         case COPY_ACL: {
             String cubeId = (String) opt.params[0];
             String modelId = (String) opt.params[1];
-            HTableInterface destAclHtable = null;
+            Table destAclHtable = null;
             try {
-                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+                destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
 
                 destAclHtable.delete(new Delete(Bytes.toBytes(cubeId)));
                 destAclHtable.delete(new Delete(Bytes.toBytes(modelId)));
-                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(destAclHtable);
             }
@@ -558,7 +558,7 @@ public class CubeMigrationCLI {
         }
     }
 
-    private static void updateMeta(KylinConfig config){
+    private static void updateMeta(KylinConfig config) {
         String[] nodes = config.getRestServers();
         for (String node : nodes) {
             RestClient restClient = new RestClient(node);

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
index 295750a..efda4e4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
@@ -26,10 +26,10 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.cube.CubeInstance;
@@ -60,7 +60,7 @@ public class CubeMigrationCheckCLI {
     private static final Option OPTION_CUBE = OptionBuilder.withArgName("cube").hasArg().isRequired(false).withDescription("The name of cube migrated").create("cube");
 
     private KylinConfig dstCfg;
-    private HBaseAdmin hbaseAdmin;
+    private Admin hbaseAdmin;
 
     private List<String> issueExistHTables;
     private List<String> inconsistentHTables;
@@ -128,9 +128,8 @@ public class CubeMigrationCheckCLI {
         this.dstCfg = kylinConfig;
         this.ifFix = isFix;
 
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        hbaseAdmin = new HBaseAdmin(conf);
-
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        hbaseAdmin = conn.getAdmin();
         issueExistHTables = Lists.newArrayList();
         inconsistentHTables = Lists.newArrayList();
     }
@@ -187,10 +186,10 @@ public class CubeMigrationCheckCLI {
                 String[] sepNameList = segFullName.split(",");
                 HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(sepNameList[0]));
                 logger.info("Change the host of htable " + sepNameList[0] + "belonging to cube " + sepNameList[1] + " from " + desc.getValue(IRealizationConstants.HTableTag) + " to " + dstCfg.getMetadataUrlPrefix());
-                hbaseAdmin.disableTable(sepNameList[0]);
+                hbaseAdmin.disableTable(TableName.valueOf(sepNameList[0]));
                 desc.setValue(IRealizationConstants.HTableTag, dstCfg.getMetadataUrlPrefix());
-                hbaseAdmin.modifyTable(sepNameList[0], desc);
-                hbaseAdmin.enableTable(sepNameList[0]);
+                hbaseAdmin.modifyTable(TableName.valueOf(sepNameList[0]), desc);
+                hbaseAdmin.enableTable(TableName.valueOf(sepNameList[0]));
             }
         } else {
             logger.info("------ Inconsistent HTables Needed To Be Fixed ------");

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
index a1193e7..48ab67c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
@@ -40,7 +40,8 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.KylinVersion;
@@ -76,7 +77,8 @@ public class DeployCoprocessorCLI {
         KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
 
         String localCoprocessorJar;
         if ("default".equals(args[0])) {
@@ -159,10 +161,10 @@ public class DeployCoprocessorCLI {
     public static void deployCoprocessor(HTableDescriptor tableDesc) {
         try {
             initHTableCoprocessor(tableDesc);
-            logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
+            logger.info("hbase table " + tableDesc.getTableName() + " deployed with coprocessor.");
 
         } catch (Exception ex) {
-            logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
+            logger.error("Error deploying coprocessor on " + tableDesc.getTableName(), ex);
             logger.error("Will try creating the table without coprocessor.");
         }
     }
@@ -184,9 +186,9 @@ public class DeployCoprocessorCLI {
         desc.addCoprocessor(CubeObserverClass, hdfsCoprocessorJar, 1002, null);
     }
 
-    public static void resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+    public static void resetCoprocessor(String tableName, Admin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
         logger.info("Disable " + tableName);
-        hbaseAdmin.disableTable(tableName);
+        hbaseAdmin.disableTable(TableName.valueOf(tableName));
 
         logger.info("Unset coprocessor on " + tableName);
         HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
@@ -214,13 +216,13 @@ public class DeployCoprocessorCLI {
             desc.setValue(IRealizationConstants.HTableGitTag, commitInfo);
         }
 
-        hbaseAdmin.modifyTable(tableName, desc);
+        hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
 
         logger.info("Enable " + tableName);
-        hbaseAdmin.enableTable(tableName);
+        hbaseAdmin.enableTable(TableName.valueOf(tableName));
     }
 
-    private static List<String> resetCoprocessorOnHTables(HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+    private static List<String> resetCoprocessorOnHTables(Admin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
         List<String> processed = new ArrayList<String>();
 
         for (String tableName : tableNames) {
@@ -331,7 +333,7 @@ public class DeployCoprocessorCLI {
         return coprocessorDir;
     }
 
-    private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
+    private static Set<String> getCoprocessorJarPaths(Admin hbaseAdmin, List<String> tableNames) throws IOException {
         HashSet<String> result = new HashSet<String>();
 
         for (String tableName : tableNames) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
index a5a85fa..29ca7b2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
@@ -26,10 +26,11 @@ import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
@@ -231,9 +232,9 @@ public class ExtendCubeToHybridCLI {
         Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
         ProjectInstance project = store.getResource(projectResPath, ProjectInstance.class, projectSerializer);
         String projUUID = project.getUuid();
-        HTableInterface aclHtable = null;
+        Table aclHtable = null;
         try {
-            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(kylinConfig.getMetadataUrlPrefix() + "_acl");
+            aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(TableName.valueOf(kylinConfig.getMetadataUrlPrefix() + "_acl"));
 
             // cube acl
             Result result = aclHtable.get(new Get(Bytes.toBytes(origCubeId)));
@@ -253,7 +254,6 @@ public class ExtendCubeToHybridCLI {
                     aclHtable.put(put);
                 }
             }
-            aclHtable.flushCommits();
         } finally {
             IOUtils.closeQuietly(aclHtable);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
index 86ba22f..dd5f8fa 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
@@ -28,13 +28,13 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Pair;
@@ -75,7 +75,7 @@ public class GridTableHBaseBenchmark {
         System.out.println("Testing grid table scanning, hit ratio " + hitRatio + ", index ratio " + indexRatio);
         String hbaseUrl = "hbase"; // use hbase-site.xml on classpath
 
-        HConnection conn = HBaseConnection.get(hbaseUrl);
+        Connection conn = HBaseConnection.get(hbaseUrl);
         createHTableIfNeeded(conn, TEST_TABLE);
         prepareData(conn);
 
@@ -91,10 +91,10 @@ public class GridTableHBaseBenchmark {
 
     }
 
-    private static void testColumnScan(HConnection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
+    private static void testColumnScan(Connection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
         Stats stats = new Stats("COLUMN_SCAN");
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -122,20 +122,20 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void testRowScanNoIndexFullScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexFullScan(Connection conn, boolean[] hits) throws IOException {
         fullScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_FULL"));
     }
 
-    private static void testRowScanNoIndexSkipScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexSkipScan(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_SKIP"));
     }
 
-    private static void testRowScanWithIndex(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanWithIndex(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_IDX"));
     }
 
-    private static void fullScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void fullScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -156,11 +156,11 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void jumpScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
+    private static void jumpScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
 
         final int jumpThreshold = 6; // compensate for Scan() overhead, totally by experience
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
 
             stats.markStart();
@@ -204,8 +204,8 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void prepareData(HConnection conn) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void prepareData(Connection conn) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
 
         try {
             // check how many rows existing
@@ -258,8 +258,8 @@ public class GridTableHBaseBenchmark {
         return bytes;
     }
 
-    private static void createHTableIfNeeded(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    private static void createHTableIfNeeded(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
             boolean tableExist = false;

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
index 6749d6c..940d64a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
@@ -24,9 +24,11 @@ import java.util.List;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -55,8 +57,8 @@ public class HBaseClean extends AbstractApplication {
     private void cleanUp() {
         try {
             // get all kylin hbase tables
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            Admin hbaseAdmin = conn.getAdmin();
             String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
             HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
             List<String> allTablesNeedToBeDropped = Lists.newArrayList();
@@ -71,12 +73,12 @@ public class HBaseClean extends AbstractApplication {
                 // drop tables
                 for (String htableName : allTablesNeedToBeDropped) {
                     logger.info("Deleting HBase table " + htableName);
-                    if (hbaseAdmin.tableExists(htableName)) {
-                        if (hbaseAdmin.isTableEnabled(htableName)) {
-                            hbaseAdmin.disableTable(htableName);
+                    if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                        if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+                            hbaseAdmin.disableTable(TableName.valueOf(htableName));
                         }
 
-                        hbaseAdmin.deleteTable(htableName);
+                        hbaseAdmin.deleteTable(TableName.valueOf(htableName));
                         logger.info("Deleted HBase table " + htableName);
                     } else {
                         logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
index 346c3a2..3ae411b 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
@@ -30,12 +31,15 @@ import java.util.TreeSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.kylin.common.util.Pair;
 import org.slf4j.Logger;
@@ -57,30 +61,31 @@ public class HBaseRegionSizeCalculator {
     /**
      * Computes size of each region for table and given column families.
      * */
-    public HBaseRegionSizeCalculator(HTable table) throws IOException {
-        this(table, new HBaseAdmin(table.getConfiguration()));
-    }
-
-    /** Constructor for unit testing */
-    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
+    public HBaseRegionSizeCalculator(String tableName, Connection hbaseConnection) throws IOException {
 
+        Table table = null;
+        Admin admin = null;
         try {
+            table = hbaseConnection.getTable(TableName.valueOf(tableName));
+            admin = hbaseConnection.getAdmin();
+
             if (!enabled(table.getConfiguration())) {
                 logger.info("Region size calculation disabled.");
                 return;
             }
 
-            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+            logger.info("Calculating region sizes for table \"" + table.getName() + "\".");
 
             // Get regions for table.
-            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+            RegionLocator regionLocator = hbaseConnection.getRegionLocator(table.getName());
+            List<HRegionLocation> regionLocationList = regionLocator.getAllRegionLocations();
             Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
 
-            for (HRegionInfo regionInfo : tableRegionInfos) {
-                tableRegions.add(regionInfo.getRegionName());
+            for (HRegionLocation hRegionLocation : regionLocationList) {
+                tableRegions.add(hRegionLocation.getRegionInfo().getRegionName());
             }
 
-            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+            ClusterStatus clusterStatus = admin.getClusterStatus();
             Collection<ServerName> servers = clusterStatus.getServers();
             final long megaByte = 1024L * 1024L;
 
@@ -104,7 +109,7 @@ public class HBaseRegionSizeCalculator {
                 }
             }
         } finally {
-            hBaseAdmin.close();
+            admin.close();
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
index 266f7e7..a2f60d4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
@@ -23,9 +23,10 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 
@@ -42,8 +43,8 @@ public class HBaseUsage {
         Map<String, List<String>> envs = Maps.newHashMap();
 
         // get all kylin hbase tables
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         for (HTableDescriptor desc : tableDescriptors) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
index f30f2c9..542df39 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
@@ -31,15 +31,15 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
@@ -57,11 +57,11 @@ public class HbaseStreamingInput {
     private static final byte[] QN = "C".getBytes();
 
     public static void createTable(String tableName) throws IOException {
-        HConnection conn = getConnection();
-        HBaseAdmin hadmin = new HBaseAdmin(conn);
+        Connection conn = getConnection();
+        Admin hadmin = conn.getAdmin();
 
         try {
-            boolean tableExist = hadmin.tableExists(tableName);
+            boolean tableExist = hadmin.tableExists(TableName.valueOf(tableName));
             if (tableExist) {
                 logger.info("HTable '" + tableName + "' already exists");
                 return;
@@ -118,8 +118,8 @@ public class HbaseStreamingInput {
                 e.printStackTrace();
             }
 
-            HConnection conn = getConnection();
-            HTableInterface table = conn.getTable(tableName);
+            Connection conn = getConnection();
+            Table table = conn.getTable(TableName.valueOf(tableName));
 
             byte[] key = new byte[8 + 4];//time + id
 
@@ -134,7 +134,7 @@ public class HbaseStreamingInput {
                 Bytes.putInt(key, 8, i);
                 Put put = new Put(key);
                 byte[] cell = randomBytes(CELL_SIZE);
-                put.add(CF, QN, cell);
+                put.addColumn(CF, QN, cell);
                 buffer.add(put);
             }
             table.put(buffer);
@@ -169,8 +169,8 @@ public class HbaseStreamingInput {
             }
 
             Random r = new Random();
-            HConnection conn = getConnection();
-            HTableInterface table = conn.getTable(tableName);
+            Connection conn = getConnection();
+            Table table = conn.getTable(TableName.valueOf(tableName));
 
             long leftBound = getFirstKeyTime(table);
             long rightBound = System.currentTimeMillis();
@@ -205,7 +205,7 @@ public class HbaseStreamingInput {
         }
     }
 
-    private static long getFirstKeyTime(HTableInterface table) throws IOException {
+    private static long getFirstKeyTime(Table table) throws IOException {
         long startTime = 0;
 
         Scan scan = new Scan();
@@ -223,8 +223,8 @@ public class HbaseStreamingInput {
 
     }
 
-    private static HConnection getConnection() throws IOException {
-        return HConnectionManager.createConnection(HBaseConnection.getCurrentHBaseConfiguration());
+    private static Connection getConnection() throws IOException {
+        return HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
     }
 
     private static String formatTime(long time) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
index ca1a060..ea05ab2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
@@ -23,10 +23,11 @@ import java.io.IOException;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.engine.mr.common.BatchConstants;
@@ -50,8 +51,8 @@ public class HtableAlterMetadataCLI extends AbstractApplication {
     String metadataValue;
 
     private void alter() throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         HTableDescriptor table = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
 
         hbaseAdmin.disableTable(table.getTableName());

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
index 8ff5b0f..df4e912 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
@@ -30,10 +30,14 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,9 +56,9 @@ public class OrphanHBaseCleanJob extends AbstractApplication {
     Set<String> metastoreWhitelistSet = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
 
     private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
-
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
         // get all kylin hbase tables
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -73,12 +77,13 @@ public class OrphanHBaseCleanJob extends AbstractApplication {
             // drop tables
             for (String htableName : allTablesNeedToBeDropped) {
                 logger.info("Deleting HBase table " + htableName);
-                if (hbaseAdmin.tableExists(htableName)) {
-                    if (hbaseAdmin.isTableEnabled(htableName)) {
-                        hbaseAdmin.disableTable(htableName);
+                TableName tableName = TableName.valueOf(htableName);
+                if (hbaseAdmin.tableExists(tableName)) {
+                    if (hbaseAdmin.isTableEnabled(tableName)) {
+                        hbaseAdmin.disableTable(tableName);
                     }
 
-                    hbaseAdmin.deleteTable(htableName);
+                    hbaseAdmin.deleteTable(tableName);
                     logger.info("Deleted HBase table " + htableName);
                 } else {
                     logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
index 58ef7cb..3403657 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
@@ -21,12 +21,13 @@ package org.apache.kylin.storage.hbase.util;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.token.TokenUtil;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -57,12 +58,12 @@ public class PingHBaseCLI {
         Scan scan = new Scan();
         int limit = 20;
 
-        HConnection conn = null;
-        HTableInterface table = null;
+        Connection conn = null;
+        Table table = null;
         ResultScanner scanner = null;
         try {
-            conn = HConnectionManager.createConnection(hconf);
-            table = conn.getTable(hbaseTable);
+            conn = ConnectionFactory.createConnection(hconf);
+            table = conn.getTable(TableName.valueOf(hbaseTable));
             scanner = table.getScanner(scan);
             int count = 0;
             for (Result r : scanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
index 01edb1f..db516bb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
@@ -22,11 +22,12 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.storage.hbase.HBaseConnection;
@@ -70,8 +71,8 @@ public class RowCounterCLI {
 
         logger.info("My Scan " + scan.toString());
 
-        HConnection conn = HConnectionManager.createConnection(conf);
-        HTableInterface tableInterface = conn.getTable(htableName);
+        Connection conn = ConnectionFactory.createConnection(conf);
+        Table tableInterface = conn.getTable(TableName.valueOf(htableName));
 
         Iterator<Result> iterator = tableInterface.getScanner(scan).iterator();
         int counter = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index dffce36..b02514f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -41,7 +41,9 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.CliCommandExecutor;
@@ -56,6 +58,7 @@ import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -76,7 +79,8 @@ public class StorageCleanupJob extends AbstractApplication {
         CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
         long TIME_THREADSHOLD = KylinConfig.getInstanceFromEnv().getStorageCleanupTimeThreshold();
         // get all kylin hbase tables
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+        Admin hbaseAdmin = conn.getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -157,22 +161,22 @@ public class StorageCleanupJob extends AbstractApplication {
     }
 
     class DeleteHTableRunnable implements Callable {
-        HBaseAdmin hbaseAdmin;
+        Admin hbaseAdmin;
         String htableName;
 
-        DeleteHTableRunnable(HBaseAdmin hbaseAdmin, String htableName) {
+        DeleteHTableRunnable(Admin hbaseAdmin, String htableName) {
             this.hbaseAdmin = hbaseAdmin;
             this.htableName = htableName;
         }
 
         public Object call() throws Exception {
             logger.info("Deleting HBase table " + htableName);
-            if (hbaseAdmin.tableExists(htableName)) {
-                if (hbaseAdmin.isTableEnabled(htableName)) {
-                    hbaseAdmin.disableTable(htableName);
+            if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+                    hbaseAdmin.disableTable(TableName.valueOf(htableName));
                 }
 
-                hbaseAdmin.deleteTable(htableName);
+                hbaseAdmin.deleteTable(TableName.valueOf(htableName));
                 logger.info("Deleted HBase table " + htableName);
             } else {
                 logger.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
index e36f662..42a54c8 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
@@ -24,16 +24,18 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,14 +51,15 @@ public class UpdateHTableHostCLI {
     private List<String> errorMsgs = Lists.newArrayList();
 
     private List<String> htables;
-    private HBaseAdmin hbaseAdmin;
+    private Admin hbaseAdmin;
     private KylinConfig kylinConfig;
     private String oldHostValue;
 
     public UpdateHTableHostCLI(List<String> htables, String oldHostValue) throws IOException {
         this.htables = htables;
         this.oldHostValue = oldHostValue;
-        this.hbaseAdmin = new HBaseAdmin(HBaseConnection.getCurrentHBaseConfiguration());
+        Connection conn = ConnectionFactory.createConnection(HBaseConfiguration.create());
+        hbaseAdmin = conn.getAdmin();
         this.kylinConfig = KylinConfig.getInstanceFromEnv();
     }
 
@@ -166,9 +169,9 @@ public class UpdateHTableHostCLI {
         HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
         if (oldHostValue.equals(desc.getValue(IRealizationConstants.HTableTag))) {
             desc.setValue(IRealizationConstants.HTableTag, kylinConfig.getMetadataUrlPrefix());
-            hbaseAdmin.disableTable(tableName);
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
+            hbaseAdmin.disableTable(TableName.valueOf(tableName));
+            hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+            hbaseAdmin.enableTable(TableName.valueOf(tableName));
 
             updatedResources.add(tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
index 390930a..1196ed6 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserverTest.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.metadata.datatype.LongMutable;
 import org.apache.kylin.metadata.model.ColumnDesc;
@@ -230,15 +231,8 @@ public class AggregateRegionObserverTest {
             return nextRaw(results);
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.InternalScanner#next(java.util
-         * .List, int)
-         */
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -307,6 +301,11 @@ public class AggregateRegionObserverTest {
             return 0;
         }
 
+        @Override
+        public int getBatch() {
+            return 0;
+        }
+
         /*
          * (non-Javadoc)
          * 
@@ -323,16 +322,9 @@ public class AggregateRegionObserverTest {
             return i < input.size();
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#nextRaw(java.util
-         * .List, int)
-         */
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-            return nextRaw(result);
+        public boolean nextRaw(List<Cell> list, ScannerContext scannerContext) throws IOException {
+            return false;
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
index 1d85922..04e2e8b 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -136,7 +137,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
 
                         Put p = new Put(rk);
                         p.setDurability(Durability.SKIP_WAL);
-                        p.add(cf.getBytes(), cq, Bytes.toBytes(c));
+                        p.addColumn(cf.getBytes(), cq, Bytes.toBytes(c));
                         ht.put(p);
                     }
                 }
@@ -224,7 +225,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
         scan.addFamily(cf.getBytes());
         scan.setFilter(filter);
         List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(table.getBytes());
-        HRegion first = regions.get(0);
+        Region first = regions.get(0);
         first.getScanner(scan);
         RegionScanner scanner = first.getScanner(scan);
         List<Cell> results = new ArrayList<Cell>();


[50/50] [abbrv] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

Posted by li...@apache.org.
KYLIN-1528 Create a branch for v1.5 with HBase 1.x API


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 787b0aad70fc7d69a95ac49aafaddd3e60c9721c
Parents: 05afca7
Author: shaofengshi <sh...@apache.org>
Authored: Wed Mar 23 17:07:05 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Tue Sep 27 08:24:32 2016 +0800

----------------------------------------------------------------------
 examples/test_case_data/sandbox/hbase-site.xml  | 19 +---
 .../kylin/provision/BuildCubeWithEngine.java    | 13 ++-
 pom.xml                                         | 12 +--
 .../kylin/rest/security/AclHBaseStorage.java    |  4 +-
 .../rest/security/MockAclHBaseStorage.java      |  8 +-
 .../apache/kylin/rest/security/MockHTable.java  | 95 ++++----------------
 .../rest/security/RealAclHBaseStorage.java      |  9 +-
 .../apache/kylin/rest/service/AclService.java   | 25 +++---
 .../apache/kylin/rest/service/CubeService.java  | 36 +++-----
 .../apache/kylin/rest/service/QueryService.java | 24 +++--
 .../apache/kylin/rest/service/UserService.java  | 17 ++--
 .../kylin/storage/hbase/HBaseConnection.java    | 44 ++++-----
 .../kylin/storage/hbase/HBaseResourceStore.java | 31 +++----
 .../kylin/storage/hbase/HBaseStorage.java       |  3 +-
 .../storage/hbase/cube/SimpleHBaseStore.java    | 20 ++---
 .../hbase/cube/v1/CubeSegmentTupleIterator.java | 11 +--
 .../storage/hbase/cube/v1/CubeStorageQuery.java |  6 +-
 .../hbase/cube/v1/RegionScannerAdapter.java     | 10 ++-
 .../cube/v1/SerializedHBaseTupleIterator.java   |  4 +-
 .../observer/AggregateRegionObserver.java       |  4 +-
 .../observer/AggregationScanner.java            | 14 ++-
 .../observer/ObserverAggregationCache.java      | 10 ++-
 .../coprocessor/observer/ObserverEnabler.java   |  4 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     | 13 +--
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |  9 +-
 .../coprocessor/endpoint/CubeVisitService.java  |  4 +-
 .../storage/hbase/steps/CubeHTableUtil.java     | 16 ++--
 .../storage/hbase/steps/DeprecatedGCStep.java   | 23 ++---
 .../storage/hbase/steps/HBaseCuboidWriter.java  |  7 +-
 .../hbase/steps/HBaseStreamingOutput.java       |  9 +-
 .../kylin/storage/hbase/steps/MergeGCStep.java  | 23 ++---
 .../storage/hbase/util/CleanHtableCLI.java      | 12 +--
 .../storage/hbase/util/CubeMigrationCLI.java    | 36 ++++----
 .../hbase/util/CubeMigrationCheckCLI.java       | 17 ++--
 .../hbase/util/DeployCoprocessorCLI.java        | 22 ++---
 .../hbase/util/ExtendCubeToHybridCLI.java       |  8 +-
 .../hbase/util/GridTableHBaseBenchmark.java     | 34 +++----
 .../kylin/storage/hbase/util/HBaseClean.java    | 18 ++--
 .../hbase/util/HBaseRegionSizeCalculator.java   | 35 ++++----
 .../kylin/storage/hbase/util/HBaseUsage.java    |  9 +-
 .../storage/hbase/util/HbaseStreamingInput.java | 30 +++----
 .../hbase/util/HtableAlterMetadataCLI.java      |  9 +-
 .../storage/hbase/util/OrphanHBaseCleanJob.java | 19 ++--
 .../kylin/storage/hbase/util/PingHBaseCLI.java  | 15 ++--
 .../kylin/storage/hbase/util/RowCounterCLI.java | 11 +--
 .../storage/hbase/util/StorageCleanupJob.java   | 20 +++--
 .../storage/hbase/util/UpdateHTableHostCLI.java | 17 ++--
 .../observer/AggregateRegionObserverTest.java   | 26 ++----
 .../v1/filter/TestFuzzyRowFilterV2EndToEnd.java |  5 +-
 49 files changed, 408 insertions(+), 462 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/examples/test_case_data/sandbox/hbase-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hbase-site.xml b/examples/test_case_data/sandbox/hbase-site.xml
index 46d5345..734908e 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -190,22 +190,5 @@
         <name>zookeeper.znode.parent</name>
         <value>/hbase-unsecure</value>
     </property>
-    <property>
-        <name>hbase.client.pause</name>
-        <value>100</value>
-        <description>General client pause value.  Used mostly as value to wait
-            before running a retry of a failed get, region lookup, etc.
-            See hbase.client.retries.number for description of how we backoff from
-            this initial pause amount and how this pause works w/ retries.</description>
-    </property>
-    <property>
-        <name>hbase.client.retries.number</name>
-        <value>5</value>
-        <description>Maximum retries.  Used as maximum for all retryable
-            operations such as the getting of a cell's value, starting a row update,
-            etc.  Retry interval is a rough function based on hbase.client.pause.  At
-            first we retry at this interval but then with backoff, we pretty quickly reach
-            retrying every ten seconds.  See HConstants#RETRY_BACKOFF for how the backup
-            ramps up.  Change this setting and hbase.client.pause to suit your workload.</description>
-    </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
index 3d60a3c..0910df5 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
@@ -35,8 +35,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
@@ -55,6 +54,7 @@ import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
 import org.apache.kylin.job.manager.ExecutableManager;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
 import org.apache.kylin.storage.hbase.util.StorageCleanupJob;
 import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
@@ -419,10 +419,10 @@ public class BuildCubeWithEngine {
     }
 
     private void checkHFilesInHBase(CubeSegment segment) throws IOException {
-        Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
-        String tableName = segment.getStorageLocationIdentifier();
-        try (HTable table = new HTable(conf, tableName)) {
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
+        try (Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl())) {
+            String tableName = segment.getStorageLocationIdentifier();
+
+            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
             Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
             long totalSize = 0;
             for (Long size : sizeMap.values()) {
@@ -448,5 +448,4 @@ public class BuildCubeWithEngine {
             }
         }
     }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1fdf81a..9808143 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,20 +46,20 @@
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
         <!-- Hadoop versions -->
-        <hadoop2.version>2.6.0</hadoop2.version>
-        <yarn.version>2.6.0</yarn.version>
+        <hadoop2.version>2.7.1</hadoop2.version>
+        <yarn.version>2.7.1</yarn.version>
 
         <!-- Hive versions -->
-        <hive.version>0.14.0</hive.version>
-        <hive-hcatalog.version>0.14.0</hive-hcatalog.version>
+        <hive.version>1.2.1</hive.version>
+        <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
 
         <!-- HBase versions -->
-        <hbase-hadoop2.version>0.98.8-hadoop2</hbase-hadoop2.version>
+        <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
         <kafka.version>0.8.1</kafka.version>
 
         <!-- Hadoop deps, keep compatible with hadoop2.version -->
         <zookeeper.version>3.4.6</zookeeper.version>
-        <curator.version>2.6.0</curator.version>
+        <curator.version>2.7.1</curator.version>
         <jackson.version>2.2.4</jackson.version>
         <jsr305.version>3.0.1</jsr305.version>
         <guava.version>14.0</guava.version>

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
index 38f299e..bfb5fe4 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
@@ -20,7 +20,7 @@ package org.apache.kylin.rest.security;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 
 /**
  */
@@ -37,6 +37,6 @@ public interface AclHBaseStorage {
 
     String prepareHBaseTable(Class<?> clazz) throws IOException;
 
-    HTableInterface getTable(String tableName) throws IOException;
+    Table getTable(String tableName) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
index d9326f5..cc76b87 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
@@ -21,7 +21,7 @@ package org.apache.kylin.rest.security;
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.QueryService;
@@ -34,8 +34,8 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
     private static final String aclTableName = "MOCK-ACL-TABLE";
     private static final String userTableName = "MOCK-USER-TABLE";
 
-    private HTableInterface mockedAclTable;
-    private HTableInterface mockedUserTable;
+    private Table mockedAclTable;
+    private Table mockedUserTable;
     private RealAclHBaseStorage realAcl;
 
     public MockAclHBaseStorage() {
@@ -65,7 +65,7 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
     }
 
     @Override
-    public HTableInterface getTable(String tableName) throws IOException {
+    public Table getTable(String tableName) throws IOException {
         if (realAcl != null) {
             return realAcl.getTable(tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
index d0aa0ed..972eea9 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
@@ -91,7 +91,7 @@ import com.google.protobuf.ServiceException;
  *     <li>remove some methods for loading data, checking values ...</li>
  * </ul>
  */
-public class MockHTable implements HTableInterface {
+public class MockHTable implements Table {
     private final String tableName;
     private final List<String> columnFamilies = new ArrayList<>();
 
@@ -114,14 +114,6 @@ public class MockHTable implements HTableInterface {
         this.columnFamilies.add(columnFamily);
     }
 
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public byte[] getTableName() {
-        return tableName.getBytes();
-    }
-
     @Override
     public TableName getName() {
         return null;
@@ -200,8 +192,8 @@ public class MockHTable implements HTableInterface {
     }
 
     @Override
-    public Boolean[] exists(List<Get> gets) throws IOException {
-        return new Boolean[0];
+    public boolean[] existsAll(List<Get> list) throws IOException {
+        return new boolean[0];
     }
 
     /**
@@ -306,15 +298,6 @@ public class MockHTable implements HTableInterface {
      * {@inheritDoc}
      */
     @Override
-    public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
-        // FIXME: implement
-        return null;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
     public ResultScanner getScanner(Scan scan) throws IOException {
         final List<Result> ret = new ArrayList<Result>();
         byte[] st = scan.getStartRow();
@@ -446,7 +429,7 @@ public class MockHTable implements HTableInterface {
              */
         }
         if (filter.hasFilterRow() && !filteredOnRowKey) {
-            filter.filterRow(nkvs);
+            filter.filterRow();
         }
         if (filter.filterRow() || filteredOnRowKey) {
             nkvs.clear();
@@ -535,6 +518,11 @@ public class MockHTable implements HTableInterface {
         return false;
     }
 
+    @Override
+    public boolean checkAndPut(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Put put) throws IOException {
+        return false;
+    }
+
     /**
      * {@inheritDoc}
      */
@@ -555,7 +543,7 @@ public class MockHTable implements HTableInterface {
                 continue;
             }
             for (KeyValue kv : delete.getFamilyMap().get(family)) {
-                if (kv.isDeleteFamily()) {
+                if (kv.isDelete()) {
                     data.get(row).get(kv.getFamily()).clear();
                 } else {
                     data.get(row).get(kv.getFamily()).remove(kv.getQualifier());
@@ -592,6 +580,11 @@ public class MockHTable implements HTableInterface {
         return false;
     }
 
+    @Override
+    public boolean checkAndDelete(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Delete delete) throws IOException {
+        return false;
+    }
+
     /**
      * {@inheritDoc}
      */
@@ -605,7 +598,7 @@ public class MockHTable implements HTableInterface {
      */
     @Override
     public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException {
-        return incrementColumnValue(row, family, qualifier, amount, true);
+        return incrementColumnValue(row, family, qualifier, amount, null);
     }
 
     @Override
@@ -617,37 +610,6 @@ public class MockHTable implements HTableInterface {
      * {@inheritDoc}
      */
     @Override
-    public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
-        if (check(row, family, qualifier, null)) {
-            Put put = new Put(row);
-            put.add(family, qualifier, Bytes.toBytes(amount));
-            put(put);
-            return amount;
-        }
-        long newValue = Bytes.toLong(data.get(row).get(family).get(qualifier).lastEntry().getValue()) + amount;
-        data.get(row).get(family).get(qualifier).put(System.currentTimeMillis(), Bytes.toBytes(newValue));
-        return newValue;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public boolean isAutoFlush() {
-        return true;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public void flushCommits() throws IOException {
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
     public void close() throws IOException {
     }
 
@@ -673,29 +635,6 @@ public class MockHTable implements HTableInterface {
      * {@inheritDoc}
      */
     @Override
-    public void setAutoFlush(boolean autoFlush) {
-        throw new NotImplementedException();
-
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
-        throw new NotImplementedException();
-
-    }
-
-    @Override
-    public void setAutoFlushTo(boolean autoFlush) {
-        throw new NotImplementedException();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
     public long getWriteBufferSize() {
         throw new NotImplementedException();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
index ab18029..d55edc3 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
@@ -21,7 +21,8 @@ package org.apache.kylin.rest.security;
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.rest.service.AclService;
 import org.apache.kylin.rest.service.QueryService;
@@ -57,11 +58,11 @@ public class RealAclHBaseStorage implements AclHBaseStorage {
     }
 
     @Override
-    public HTableInterface getTable(String tableName) throws IOException {
+    public Table getTable(String tableName) throws IOException {
         if (StringUtils.equals(tableName, aclTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
         } else if (StringUtils.equals(tableName, userTableName)) {
-            return HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
         } else {
             throw new IllegalStateException("getTable failed" + tableName);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java b/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
index d693a67..3e3efec 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
@@ -33,7 +33,7 @@ import javax.annotation.PostConstruct;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
@@ -124,7 +124,7 @@ public class AclService implements MutableAclService {
     @Override
     public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
         List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -173,7 +173,7 @@ public class AclService implements MutableAclService {
     @Override
     public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
         Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
-        HTableInterface htable = null;
+        Table htable = null;
         Result result = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
@@ -226,17 +226,16 @@ public class AclService implements MutableAclService {
         Authentication auth = SecurityContextHolder.getContext().getAuthentication();
         PrincipalSid sid = new PrincipalSid(auth);
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
             Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
-            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
-            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
-            put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
 
             htable.put(put);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " created successfully.");
         } catch (IOException e) {
@@ -250,7 +249,7 @@ public class AclService implements MutableAclService {
 
     @Override
     public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -266,7 +265,6 @@ public class AclService implements MutableAclService {
             }
 
             htable.delete(delete);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " deleted successfully.");
         } catch (IOException e) {
@@ -284,7 +282,7 @@ public class AclService implements MutableAclService {
             throw e;
         }
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(aclTableName);
 
@@ -295,17 +293,16 @@ public class AclService implements MutableAclService {
             Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
 
             if (null != acl.getParentAcl()) {
-                put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+                put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
             }
 
             for (AccessControlEntry ace : acl.getEntries()) {
                 AceInfo aceInfo = new AceInfo(ace);
-                put.add(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+                put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
             }
 
             if (!put.isEmpty()) {
                 htable.put(put);
-                htable.flushCommits();
 
                 logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
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 e446045..c5e4c47 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
@@ -28,8 +28,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.WeakHashMap;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeInstance;
@@ -409,35 +408,24 @@ public class CubeService extends BasicService {
         if (htableInfoCache.containsKey(tableName)) {
             return htableInfoCache.get(tableName);
         }
-
-        Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
-        HTable table = null;
+        Connection conn = HBaseConnection.get(this.getConfig().getStorageUrl());
         HBaseResponse hr = null;
         long tableSize = 0;
         int regionCount = 0;
 
-        try {
-            table = new HTable(hconf, tableName);
-
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
-            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
-
-            for (long s : sizeMap.values()) {
-                tableSize += s;
-            }
-
-            regionCount = sizeMap.size();
+        HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
+        Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
 
-            // Set response.
-            hr = new HBaseResponse();
-            hr.setTableSize(tableSize);
-            hr.setRegionCount(regionCount);
-        } finally {
-            if (null != table) {
-                table.close();
-            }
+        for (long s : sizeMap.values()) {
+            tableSize += s;
         }
 
+        regionCount = sizeMap.size();
+
+        // Set response.
+        hr = new HBaseResponse();
+        hr.setTableSize(tableSize);
+        hr.setRegionCount(regionCount);
         htableInfoCache.put(tableName, hr);
 
         return hr;

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 8303cee..9c900c0 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -45,11 +45,11 @@ import javax.sql.DataSource;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.CubeInstance;
@@ -137,14 +137,13 @@ public class QueryService extends BasicService {
         Query[] queryArray = new Query[queries.size()];
 
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -170,14 +169,13 @@ public class QueryService extends BasicService {
 
         Query[] queryArray = new Query[queries.size()];
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -189,12 +187,12 @@ public class QueryService extends BasicService {
         }
 
         List<Query> queries = new ArrayList<Query>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            HConnection conn = HBaseConnection.get(hbaseUrl);
+            org.apache.hadoop.hbase.client.Connection conn = HBaseConnection.get(hbaseUrl);
             HBaseConnection.createHTableIfNeeded(conn, userTableName, USER_QUERY_FAMILY);
 
-            htable = conn.getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Get get = new Get(Bytes.toBytes(creator));
             get.addFamily(Bytes.toBytes(USER_QUERY_FAMILY));
             Result result = htable.get(get);

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
index 07c7c6f..ab54882 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
@@ -30,11 +30,11 @@ import javax.annotation.PostConstruct;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.rest.security.AclHBaseStorage;
@@ -72,7 +72,7 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public UserDetails loadUserByUsername(String username) throws UsernameNotFoundException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
@@ -144,16 +144,16 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public void updateUser(UserDetails user) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
             Pair<byte[], byte[]> pair = userToHBaseRow(user);
             Put put = new Put(pair.getKey());
-            put.add(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
+
+            put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
 
             htable.put(put);
-            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -163,14 +163,13 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public void deleteUser(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
             Delete delete = new Delete(Bytes.toBytes(username));
 
             htable.delete(delete);
-            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -185,7 +184,7 @@ public class UserService implements UserDetailsManager {
 
     @Override
     public boolean userExists(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);
 
@@ -216,7 +215,7 @@ public class UserService implements UserDetailsManager {
         s.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN));
 
         List<UserDetails> all = new ArrayList<UserDetails>();
-        HTableInterface htable = null;
+        Table htable = null;
         ResultScanner scanner = null;
         try {
             htable = aclHBaseStorage.getTable(userTableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
index cbf81b6..b769391 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
@@ -40,9 +40,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.kylin.common.KylinConfig;
@@ -64,7 +64,7 @@ public class HBaseConnection {
     private static final Logger logger = LoggerFactory.getLogger(HBaseConnection.class);
 
     private static final Map<String, Configuration> configCache = new ConcurrentHashMap<String, Configuration>();
-    private static final Map<String, HConnection> connPool = new ConcurrentHashMap<String, HConnection>();
+    private static final Map<String, Connection> connPool = new ConcurrentHashMap<String, Connection>();
     private static final ThreadLocal<Configuration> configThreadLocal = new ThreadLocal<>();
 
     private static ExecutorService coprocessorPool = null;
@@ -75,7 +75,7 @@ public class HBaseConnection {
             public void run() {
                 closeCoprocessorPool();
 
-                for (HConnection conn : connPool.values()) {
+                for (Connection conn : connPool.values()) {
                     try {
                         conn.close();
                     } catch (IOException e) {
@@ -144,7 +144,7 @@ public class HBaseConnection {
         // using a hbase:xxx URL is deprecated, instead hbase config is always loaded from hbase-site.xml in classpath
         if (!(StringUtils.isEmpty(url) || "hbase".equals(url)))
             throw new IllegalArgumentException("to use hbase storage, pls set 'kylin.storage.url=hbase' in kylin.properties");
-        
+
         Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
         addHBaseClusterNNHAConfiguration(conf);
 
@@ -213,9 +213,9 @@ public class HBaseConnection {
 
     // ============================================================================
 
-    // returned HConnection can be shared by multiple threads and does not require close()
+    // returned Connection can be shared by multiple threads and does not require close()
     @SuppressWarnings("resource")
-    public static HConnection get(String url) {
+    public static Connection get(String url) {
         // find configuration
         Configuration conf = configCache.get(url);
         if (conf == null) {
@@ -223,13 +223,13 @@ public class HBaseConnection {
             configCache.put(url, conf);
         }
 
-        HConnection connection = connPool.get(url);
+        Connection connection = connPool.get(url);
         try {
             while (true) {
                 // I don't use DCL since recreate a connection is not a big issue.
                 if (connection == null || connection.isClosed()) {
                     logger.info("connection is null or closed, creating a new one");
-                    connection = HConnectionManager.createConnection(conf);
+                    connection = ConnectionFactory.createConnection(conf);
                     connPool.put(url, connection);
                 }
 
@@ -248,8 +248,8 @@ public class HBaseConnection {
         return connection;
     }
 
-    public static boolean tableExists(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    public static boolean tableExists(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
         try {
             return hbase.tableExists(TableName.valueOf(tableName));
         } finally {
@@ -269,18 +269,18 @@ public class HBaseConnection {
         deleteTable(HBaseConnection.get(hbaseUrl), tableName);
     }
 
-    public static void createHTableIfNeeded(HConnection conn, String table, String... families) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
-
+    public static void createHTableIfNeeded(Connection conn, String table, String... families) throws IOException {
+        Admin hbase = conn.getAdmin();
+        TableName tableName = TableName.valueOf(table);
         try {
             if (tableExists(conn, table)) {
                 logger.debug("HTable '" + table + "' already exists");
-                Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(TableName.valueOf(table)));
+                Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(tableName));
                 boolean wait = false;
                 for (String family : families) {
                     if (existingFamilies.contains(family) == false) {
                         logger.debug("Adding family '" + family + "' to HTable '" + table + "'");
-                        hbase.addColumn(table, newFamilyDescriptor(family));
+                        hbase.addColumn(tableName, newFamilyDescriptor(family));
                         // addColumn() is async, is there a way to wait it finish?
                         wait = true;
                     }
@@ -333,8 +333,8 @@ public class HBaseConnection {
         return fd;
     }
 
-    public static void deleteTable(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    public static void deleteTable(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
             if (!tableExists(conn, tableName)) {
@@ -344,10 +344,10 @@ public class HBaseConnection {
 
             logger.debug("delete HTable '" + tableName + "'");
 
-            if (hbase.isTableEnabled(tableName)) {
-                hbase.disableTable(tableName);
+            if (hbase.isTableEnabled(TableName.valueOf(tableName))) {
+                hbase.disableTable(TableName.valueOf(tableName));
             }
-            hbase.deleteTable(tableName);
+            hbase.deleteTable(TableName.valueOf(tableName));
 
             logger.debug("HTable '" + tableName + "' deleted");
         } finally {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
index 3fd6426..169510a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
@@ -31,14 +31,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
@@ -69,7 +70,7 @@ public class HBaseResourceStore extends ResourceStore {
     final String tableNameBase;
     final String hbaseUrl;
 
-    HConnection getConnection() throws IOException {
+    Connection getConnection() throws IOException {
         return HBaseConnection.get(hbaseUrl);
     }
 
@@ -120,7 +121,7 @@ public class HBaseResourceStore extends ResourceStore {
         byte[] endRow = Bytes.toBytes(lookForPrefix);
         endRow[endRow.length - 1]++;
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         Scan scan = new Scan(startRow, endRow);
         if ((filter != null && filter instanceof KeyOnlyFilter) == false) {
             scan.addColumn(B_FAMILY, B_COLUMN_TS);
@@ -238,13 +239,12 @@ public class HBaseResourceStore extends ResourceStore {
         IOUtils.copy(content, bout);
         bout.close();
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             Put put = buildPut(resPath, ts, row, bout.toByteArray(), table);
 
             table.put(put);
-            table.flushCommits();
         } finally {
             IOUtils.closeQuietly(table);
         }
@@ -252,7 +252,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
@@ -265,8 +265,6 @@ public class HBaseResourceStore extends ResourceStore {
                 throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but it is " + real);
             }
 
-            table.flushCommits();
-
             return newTS;
         } finally {
             IOUtils.closeQuietly(table);
@@ -275,7 +273,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected void deleteResourceImpl(String resPath) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             boolean hdfsResourceExist = false;
             Result result = internalGetFromHTable(table, resPath, true, false);
@@ -288,7 +286,6 @@ public class HBaseResourceStore extends ResourceStore {
 
             Delete del = new Delete(Bytes.toBytes(resPath));
             table.delete(del);
-            table.flushCommits();
 
             if (hdfsResourceExist) { // remove hdfs cell value
                 Path redirectPath = bigCellHDFSPath(resPath);
@@ -310,7 +307,7 @@ public class HBaseResourceStore extends ResourceStore {
     }
 
     private Result getFromHTable(String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             return internalGetFromHTable(table, path, fetchContent, fetchTimestamp);
         } finally {
@@ -318,7 +315,7 @@ public class HBaseResourceStore extends ResourceStore {
         }
     }
 
-    private Result internalGetFromHTable(HTableInterface table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
+    private Result internalGetFromHTable(Table table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
         byte[] rowkey = Bytes.toBytes(path);
 
         Get get = new Get(rowkey);
@@ -337,7 +334,7 @@ public class HBaseResourceStore extends ResourceStore {
         return exists ? result : null;
     }
 
-    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
+    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {
         Path redirectPath = bigCellHDFSPath(resPath);
         Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
@@ -363,7 +360,7 @@ public class HBaseResourceStore extends ResourceStore {
         return redirectPath;
     }
 
-    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
+    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, Table table) throws IOException {
         int kvSizeLimit = Integer.parseInt(getConnection().getConfiguration().get("hbase.client.keyvalue.maxsize", "10485760"));
         if (content.length > kvSizeLimit) {
             writeLargeCellToHdfs(resPath, content, table);
@@ -371,8 +368,8 @@ public class HBaseResourceStore extends ResourceStore {
         }
 
         Put put = new Put(row);
-        put.add(B_FAMILY, B_COLUMN, content);
-        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+        put.addColumn(B_FAMILY, B_COLUMN, content);
+        put.addColumn(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
 
         return put;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
index f4dfd2b..3d82105 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseStorage.java
@@ -18,7 +18,6 @@
 
 package org.apache.kylin.storage.hbase;
 
-import com.google.common.base.Preconditions;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
 import org.apache.kylin.cube.CubeInstance;
@@ -36,6 +35,8 @@ import org.apache.kylin.storage.IStorageQuery;
 import org.apache.kylin.storage.hbase.steps.HBaseMROutput;
 import org.apache.kylin.storage.hbase.steps.HBaseMROutput2Transition;
 
+import com.google.common.base.Preconditions;
+
 @SuppressWarnings("unused")
 //used by reflection
 public class HBaseStorage implements IStorage {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
index b141190..f63d9c2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
@@ -26,12 +26,13 @@ import java.util.NoSuchElementException;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.kv.RowConstants;
@@ -86,14 +87,13 @@ public class SimpleHBaseStore implements IGTStore {
     }
 
     private class Writer implements IGTWriter {
-        final HTableInterface table;
+        final BufferedMutator table;
         final ByteBuffer rowkey = ByteBuffer.allocate(50);
         final ByteBuffer value = ByteBuffer.allocate(50);
 
         Writer() throws IOException {
-            HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
-            table = conn.getTable(htableName);
-            table.setAutoFlush(false, true);
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            table = conn.getBufferedMutator(htableName);
         }
 
         @Override
@@ -113,24 +113,24 @@ public class SimpleHBaseStore implements IGTStore {
 
             Put put = new Put(rowkey);
             put.addImmutable(CF_B, ByteBuffer.wrap(COL_B), HConstants.LATEST_TIMESTAMP, value);
-            table.put(put);
+            table.mutate(put);
         }
 
         @Override
         public void close() throws IOException {
-            table.flushCommits();
+            table.flush();
             table.close();
         }
     }
 
     class Reader implements IGTScanner {
-        final HTableInterface table;
+        final Table table;
         final ResultScanner scanner;
 
         int count = 0;
 
         Reader() throws IOException {
-            HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+            Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
             table = conn.getTable(htableName);
 
             Scan scan = new Scan();

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
index 8ac3832..982a044 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeSegmentTupleIterator.java
@@ -25,11 +25,12 @@ import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FuzzyRowFilter;
@@ -70,7 +71,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     protected final List<RowValueDecoder> rowValueDecoders;
     private final StorageContext context;
     private final String tableName;
-    private final HTableInterface table;
+    private final Table table;
 
     protected CubeTupleConverter tupleConverter;
     protected final Iterator<HBaseKeyRange> rangeIterator;
@@ -88,7 +89,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     private int advMeasureRowsRemaining;
     private int advMeasureRowIndex;
 
-    public CubeSegmentTupleIterator(CubeSegment cubeSeg, List<HBaseKeyRange> keyRanges, HConnection conn, //
+    public CubeSegmentTupleIterator(CubeSegment cubeSeg, List<HBaseKeyRange> keyRanges, Connection conn, //
             Set<TblColRef> dimensions, TupleFilter filter, Set<TblColRef> groupBy, //
             List<RowValueDecoder> rowValueDecoders, StorageContext context, TupleInfo returnTupleInfo) {
         this.cubeSeg = cubeSeg;
@@ -108,7 +109,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
         this.rangeIterator = keyRanges.iterator();
 
         try {
-            this.table = conn.getTable(tableName);
+            this.table = conn.getTable(TableName.valueOf(tableName));
         } catch (Throwable t) {
             throw new StorageException("Error when open connection to table " + tableName, t);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
index ff729f4..1944327 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/CubeStorageQuery.java
@@ -33,7 +33,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.Dictionary;
@@ -46,10 +46,10 @@ import org.apache.kylin.cube.RawQueryLastHacker;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
 import org.apache.kylin.cube.model.HBaseColumnDesc;
 import org.apache.kylin.cube.model.HBaseMappingDesc;
 import org.apache.kylin.cube.model.RowKeyDesc;
-import org.apache.kylin.cube.model.CubeDesc.DeriveInfo;
 import org.apache.kylin.dict.lookup.LookupStringTable;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
@@ -152,7 +152,7 @@ public class CubeStorageQuery implements IStorageQuery {
         setCoprocessor(groupsCopD, valueDecoders, context); // enable coprocessor if beneficial
         setLimit(filter, context);
 
-        HConnection conn = HBaseConnection.get(context.getConnUrl());
+        Connection conn = HBaseConnection.get(context.getConnUrl());
 
         // notice we're passing filterD down to storage instead of flatFilter
         return new SerializedHBaseTupleIterator(conn, scans, cubeInstance, dimensionsD, filterD, groupsCopD, valueDecoders, context, returnTupleInfo);

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
index 6342c5c..0ade920 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/RegionScannerAdapter.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
 /**
  * @author yangli9
@@ -50,7 +51,7 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
@@ -60,7 +61,7 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
@@ -94,4 +95,9 @@ public class RegionScannerAdapter implements RegionScanner {
         return Long.MAX_VALUE;
     }
 
+    @Override
+    public int getBatch() {
+        return -1;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
index e8dd5b9..d033c77 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/SerializedHBaseTupleIterator.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.metadata.filter.TupleFilter;
@@ -57,7 +57,7 @@ public class SerializedHBaseTupleIterator implements ITupleIterator {
     private int scanCount;
     private ITuple next;
 
-    public SerializedHBaseTupleIterator(HConnection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, //
+    public SerializedHBaseTupleIterator(Connection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, //
             Set<TblColRef> dimensions, TupleFilter filter, Set<TblColRef> groupBy, List<RowValueDecoder> rowValueDecoders, //
             StorageContext context, TupleInfo returnTupleInfo) {
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
index 7139ca7..7e25e4c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregateRegionObserver.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.gridtable.StorageSideBehavior;
@@ -99,7 +99,7 @@ public class AggregateRegionObserver extends BaseRegionObserver {
         // start/end region operation & sync on scanner is suggested by the
         // javadoc of RegionScanner.nextRaw()
         // FIXME: will the lock still work when a iterator is returned? is it safe? Is readonly attribute helping here? by mhb
-        HRegion region = ctxt.getEnvironment().getRegion();
+        Region region = ctxt.getEnvironment().getRegion();
         region.startRegionOperation();
         try {
             synchronized (innerScanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
index a900ea1..d64f48f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/AggregationScanner.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.gridtable.StorageSideBehavior;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
@@ -116,8 +117,8 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
-        return outerScanner.next(result, limit);
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.next(result, scannerContext);
     }
 
     @Override
@@ -126,8 +127,8 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-        return outerScanner.nextRaw(result, limit);
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.nextRaw(result, scannerContext);
     }
 
     @Override
@@ -160,6 +161,11 @@ public class AggregationScanner implements RegionScanner {
         return outerScanner.getMvccReadPoint();
     }
 
+    @Override
+    public int getBatch() {
+        return outerScanner.getBatch();
+    }
+
     private static class Stats {
         long inputRows = 0;
         long inputBytes = 0;

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
index 8404262..331e34d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverAggregationCache.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.common.coprocessor.AggregationCache;
@@ -112,7 +113,7 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -122,7 +123,7 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+        public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -161,6 +162,11 @@ public class ObserverAggregationCache extends AggregationCache {
             // AggregateRegionObserver.LOG.info("Kylin Scanner getMvccReadPoint()");
             return Long.MAX_VALUE;
         }
+
+        @Override
+        public int getBatch() {
+            return innerScanner.getBatch();
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java
index 394b3e2..9fd33f5 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v1/coprocessor/observer/ObserverEnabler.java
@@ -23,9 +23,9 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
@@ -60,7 +60,7 @@ public class ObserverEnabler {
     static final Map<String, Boolean> CUBE_OVERRIDES = Maps.newConcurrentMap();
 
     public static ResultScanner scanWithCoprocessorIfBeneficial(CubeSegment segment, Cuboid cuboid, TupleFilter tupleFiler, //
-            Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context, HTableInterface table, Scan scan) throws IOException {
+            Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context, Table table, Scan scan) throws IOException {
 
         if (context.isCoprocessorEnabled() == false) {
             return table.getScanner(scan);

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index c7de287..254541c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -26,8 +26,9 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.zip.DataFormatException;
 
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
@@ -49,10 +50,10 @@ import org.apache.kylin.storage.gtrecord.StorageResponseGTScatter;
 import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos;
 import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitRequest;
-import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse;
-import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitService;
 import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitRequest.IntList;
+import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse;
 import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats;
+import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -115,7 +116,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
         final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
 
         // globally shared connection, does not require close
-        final HConnection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+        final Connection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
 
         final List<IntList> hbaseColumnsToGTIntList = Lists.newArrayList();
         List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);
@@ -164,7 +165,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
                     final boolean[] abnormalFinish = new boolean[1];
 
                     try {
-                        HTableInterface table = conn.getTable(cubeSeg.getStorageLocationIdentifier(), HBaseConnection.getCoprocessorPool());
+                        Table table = conn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()), HBaseConnection.getCoprocessorPool());
 
                         final CubeVisitRequest request = builder.build();
                         final byte[] startKey = epRange.getFirst();

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
index f1e5dab..68c9534 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
@@ -24,11 +24,12 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.ShardingHash;
@@ -164,8 +165,8 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
         // primary key (also the 0th column block) is always selected
         final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
         // globally shared connection, does not require close
-        HConnection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
-        final HTableInterface hbaseTable = hbaseConn.getTable(cubeSeg.getStorageLocationIdentifier());
+        Connection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+        final Table hbaseTable = hbaseConn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()));
 
         List<RawScan> rawScans = preparedHBaseScans(scanRequest.getGTScanRanges(), selectedColBlocks);
         List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index 4790d6e..82ebe2e 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -145,7 +145,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         if (shardLength == 0) {
             return;
         }
-        byte[] regionStartKey = ArrayUtils.isEmpty(region.getStartKey()) ? new byte[shardLength] : region.getStartKey();
+        byte[] regionStartKey = ArrayUtils.isEmpty(region.getRegionInfo().getStartKey()) ? new byte[shardLength] : region.getRegionInfo().getStartKey();
         Bytes.putBytes(rawScan.startKey, 0, regionStartKey, 0, shardLength);
         Bytes.putBytes(rawScan.endKey, 0, regionStartKey, 0, shardLength);
     }
@@ -181,7 +181,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
         try {
             this.serviceStartTime = System.currentTimeMillis();
 
-            region = env.getRegion();
+            region = (HRegion)env.getRegion();
             region.startRegionOperation();
 
             // if user change kylin.properties on kylin server, need to manually redeploy coprocessor jar to update KylinConfig of Env.

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
index 9b487a7..4a4f2a3 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
@@ -25,7 +25,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -78,7 +79,8 @@ public class CubeHTableUtil {
         tableDesc.setValue(IRealizationConstants.HTableSegmentTag, cubeSegment.toString());
 
         Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+        Admin admin = conn.getAdmin();
 
         try {
             if (User.isHBaseSecurityEnabled(conf)) {
@@ -91,7 +93,7 @@ public class CubeHTableUtil {
                 tableDesc.addFamily(cf);
             }
 
-            if (admin.tableExists(tableName)) {
+            if (admin.tableExists(TableName.valueOf(tableName))) {
                 // admin.disableTable(tableName);
                 // admin.deleteTable(tableName);
                 throw new RuntimeException("HBase table " + tableName + " exists!");
@@ -100,7 +102,7 @@ public class CubeHTableUtil {
             DeployCoprocessorCLI.deployCoprocessor(tableDesc);
 
             admin.createTable(tableDesc, splitKeys);
-            Preconditions.checkArgument(admin.isTableAvailable(tableName), "table " + tableName + " created, but is not available due to some reasons");
+            Preconditions.checkArgument(admin.isTableAvailable(TableName.valueOf(tableName)), "table " + tableName + " created, but is not available due to some reasons");
             logger.info("create hbase table " + tableName + " done.");
         } finally {
             admin.close();
@@ -109,8 +111,7 @@ public class CubeHTableUtil {
     }
 
     public static void deleteHTable(TableName tableName) throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
         try {
             if (admin.tableExists(tableName)) {
                 logger.info("disabling hbase table " + tableName);
@@ -125,8 +126,7 @@ public class CubeHTableUtil {
 
     /** create a HTable that has the same performance settings as normal cube table, for benchmark purpose */
     public static void createBenchmarkHTable(TableName tableName, String cfName) throws IOException {
-        Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
         try {
             if (admin.tableExists(tableName)) {
                 logger.info("disabling hbase table " + tableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
index 7aecd7e..9dc9715 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
@@ -28,9 +28,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;
@@ -99,19 +100,21 @@ public class DeprecatedGCStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-            HBaseAdmin admin = null;
+            Admin admin = null;
             try {
-                admin = new HBaseAdmin(conf);
+
+                Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+                admin = conn.getAdmin();
+
                 for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                    if (admin.tableExists(TableName.valueOf(table))) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf(table));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
+                            if (admin.isTableEnabled(TableName.valueOf(table))) {
+                                admin.disableTable(TableName.valueOf(table));
                             }
-                            admin.deleteTable(table);
+                            admin.deleteTable(TableName.valueOf(table));
                             logger.debug("Dropped HBase table " + table);
                             output.append("Dropped HBase table " + table + " \n");
                         } else {

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index d5b36df..6587d4e 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -23,8 +23,8 @@ import java.util.List;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
@@ -49,7 +49,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
 
     private final List<KeyValueCreator> keyValueCreators;
     private final int nColumns;
-    private final HTableInterface hTable;
+    private final Table hTable;
     private final CubeDesc cubeDesc;
     private final CubeSegment cubeSegment;
     private final Object[] measureValues;
@@ -58,7 +58,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
     private AbstractRowKeyEncoder rowKeyEncoder;
     private byte[] keybuf;
 
-    public HBaseCuboidWriter(CubeSegment segment, HTableInterface hTable) {
+    public HBaseCuboidWriter(CubeSegment segment, Table hTable) {
         this.keyValueCreators = Lists.newArrayList();
         this.cubeSegment = segment;
         this.cubeDesc = cubeSegment.getCubeDesc();
@@ -117,7 +117,6 @@ public class HBaseCuboidWriter implements ICuboidWriter {
             long t = System.currentTimeMillis();
             if (hTable != null) {
                 hTable.put(puts);
-                hTable.flushCommits();
             }
             logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
             puts.clear();

http://git-wip-us.apache.org/repos/asf/kylin/blob/787b0aad/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java
index 9adaf24..e1e2cba 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseStreamingOutput.java
@@ -27,7 +27,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.cube.CubeSegment;
@@ -56,7 +57,7 @@ public class HBaseStreamingOutput implements IStreamingOutput {
         try {
             CubeSegment cubeSegment = (CubeSegment) buildable;
 
-            final HTableInterface hTable;
+            final Table hTable;
             hTable = createHTable(cubeSegment);
             List<ICuboidWriter> cuboidWriters = Lists.newArrayList();
             cuboidWriters.add(new HBaseCuboidWriter(cubeSegment, hTable));
@@ -88,10 +89,10 @@ public class HBaseStreamingOutput implements IStreamingOutput {
         }
     }
 
-    private HTableInterface createHTable(final CubeSegment cubeSegment) throws IOException {
+    private Table createHTable(final CubeSegment cubeSegment) throws IOException {
         final String hTableName = cubeSegment.getStorageLocationIdentifier();
         CubeHTableUtil.createHTable(cubeSegment, null);
-        final HTableInterface hTable = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getTable(hTableName);
+        final Table hTable = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getTable(TableName.valueOf(hTableName));
         logger.info("hTable:" + hTableName + " for segment:" + cubeSegment.getName() + " created!");
         return hTable;
     }


[28/50] [abbrv] kylin git commit: fix CI, remove conflicting httpclient from job jar

Posted by li...@apache.org.
fix CI, remove conflicting httpclient from job jar


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: 65afee78287186557975e57aca3843689195e1a0
Parents: 72005ea
Author: Yang Li <li...@apache.org>
Authored: Fri Sep 23 23:47:24 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Fri Sep 23 23:47:24 2016 +0800

----------------------------------------------------------------------
 assembly/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/65afee78/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index c7bde96..0c80afc 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -180,6 +180,7 @@
                                     <exclude>io.netty:*</exclude>
                                     <exclude>org.apache.zookeeper:*</exclude>
                                     <exclude>net.sf.ehcache:*</exclude>
+                                    <exclude>org.apache.httpcomponents:*</exclude>
                                 </excludes>
                             </artifactSet>
                             <filters>


[29/50] [abbrv] kylin git commit: KYLIN-2043 rollback httpclient to 4.2.5 to align with Hadoop 2.6/2.7

Posted by li...@apache.org.
KYLIN-2043 rollback httpclient to 4.2.5 to align with Hadoop 2.6/2.7


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

Branch: refs/heads/1.5.x-HBase1.x
Commit: b6cdc0d1b20683feb7255a71dfa8d5b98c4e636e
Parents: 65afee7
Author: Yang Li <li...@apache.org>
Authored: Sat Sep 24 01:31:22 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Sep 24 01:31:22 2016 +0800

----------------------------------------------------------------------
 .../kylin/common/restclient/RestClient.java     | 35 +++++++--------
 .../engine/mr/common/HadoopStatusGetter.java    | 27 ++++++------
 .../java/org/apache/kylin/jdbc/KylinClient.java | 46 ++++++++++----------
 pom.xml                                         |  2 +-
 4 files changed, 52 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b6cdc0d1/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
index 46a9e9b..363cc54 100644
--- a/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
+++ b/core-common/src/main/java/org/apache/kylin/common/restclient/RestClient.java
@@ -23,16 +23,15 @@ import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.http.auth.AuthScope;
-import org.apache.http.auth.UsernamePasswordCredentials;
-import org.apache.http.client.CredentialsProvider;
-import org.apache.http.client.methods.CloseableHttpResponse;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.impl.client.BasicCredentialsProvider;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
-import org.apache.http.util.EntityUtils;
+import org.apache.http.HttpResponse;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.util.EntityUtils;
 import org.apache.kylin.common.util.JsonUtil;
 
 /**
@@ -45,7 +44,7 @@ public class RestClient {
     protected String baseUrl;
     protected String userName;
     protected String password;
-    protected CloseableHttpClient client;
+    protected DefaultHttpClient client;
 
     protected static Pattern fullRestPattern = Pattern.compile("(?:([^:]+)[:]([^@]+)[@])?([^:]+)(?:[:](\\d+))?");
 
@@ -79,13 +78,13 @@ public class RestClient {
         this.password = password;
         this.baseUrl = "http://" + host + ":" + port + "/kylin/api";
 
-        client = HttpClients.createDefault();
+        client = new DefaultHttpClient();
 
         if (userName != null && password != null) {
             CredentialsProvider provider = new BasicCredentialsProvider();
             UsernamePasswordCredentials credentials = new UsernamePasswordCredentials(userName, password);
-            provider.setCredentials(AuthScope.ANY, credentials);
-            client = HttpClients.custom().setDefaultCredentialsProvider(provider).build();
+            provider.setCredentials(AuthScope.ANY, credentials);
+            client.setCredentialsProvider(provider);
         }
     }
 
@@ -94,12 +93,11 @@ public class RestClient {
         HttpPut request = new HttpPut(url);
 
         try {
-            CloseableHttpResponse response = client.execute(request);
+            HttpResponse response = client.execute(request);
             String msg = EntityUtils.toString(response.getEntity());
 
             if (response.getStatusLine().getStatusCode() != 200)
-                throw new IOException("Invalid response " + response.getStatusLine().getStatusCode() + " with cache wipe url " + url + "\n" + msg);
-            response.close();
+                throw new IOException("Invalid response " + response.getStatusLine().getStatusCode() + " with cache wipe url " + url + "\n" + msg);
         } catch (Exception ex) {
             throw new IOException(ex);
         } finally {
@@ -111,14 +109,13 @@ public class RestClient {
         String url = baseUrl + "/admin/config";
         HttpGet request = new HttpGet(url);
         try {
-            CloseableHttpResponse response = client.execute(request);
+            HttpResponse response = client.execute(request);
             String msg = EntityUtils.toString(response.getEntity());
             Map<String, String> map = JsonUtil.readValueAsMap(msg);
             msg = map.get("config");
 
             if (response.getStatusLine().getStatusCode() != 200)
                 throw new IOException("Invalid response " + response.getStatusLine().getStatusCode() + " with cache wipe url " + url + "\n" + msg);
-            response.close();
             return msg;
         } finally {
             request.releaseConnection();

http://git-wip-us.apache.org/repos/asf/kylin/blob/b6cdc0d1/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
index 14f7235..619de90 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/HadoopStatusGetter.java
@@ -34,19 +34,15 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
-import org.apache.http.auth.AuthSchemeProvider;
+import org.apache.http.HttpResponse;
+import org.apache.http.auth.AuthSchemeRegistry;
 import org.apache.http.auth.AuthScope;
 import org.apache.http.auth.Credentials;
-import org.apache.http.client.config.AuthSchemes;
-import org.apache.http.client.methods.CloseableHttpResponse;
 import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.protocol.HttpClientContext;
-import org.apache.http.config.Lookup;
-import org.apache.http.config.RegistryBuilder;
+import org.apache.http.client.params.AuthPolicy;
 import org.apache.http.impl.auth.SPNegoSchemeFactory;
 import org.apache.http.impl.client.BasicCredentialsProvider;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
+import org.apache.http.impl.client.DefaultHttpClient;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
@@ -88,9 +84,12 @@ public class HadoopStatusGetter {
         System.setProperty("java.security.krb5.conf", krb5ConfigPath);
         System.setProperty("sun.security.krb5.debug", "true");
         System.setProperty("javax.security.auth.useSubjectCredsOnly", "false");
-        Lookup<AuthSchemeProvider> authSchemeRegistry = RegistryBuilder.<AuthSchemeProvider> create().register(AuthSchemes.SPNEGO, new SPNegoSchemeFactory(skipPortAtKerberosDatabaseLookup)).build();
-        CloseableHttpClient client = HttpClients.custom().setDefaultAuthSchemeRegistry(authSchemeRegistry).build();
-        HttpClientContext context = HttpClientContext.create();
+        
+        DefaultHttpClient client = new DefaultHttpClient();
+        AuthSchemeRegistry authSchemeRegistry = new AuthSchemeRegistry();
+        authSchemeRegistry.register(AuthPolicy.SPNEGO, new SPNegoSchemeFactory(skipPortAtKerberosDatabaseLookup));
+        client.setAuthSchemes(authSchemeRegistry);
+        
         BasicCredentialsProvider credentialsProvider = new BasicCredentialsProvider();
         Credentials useJaasCreds = new Credentials() {
             public String getPassword() {
@@ -101,9 +100,9 @@ public class HadoopStatusGetter {
                 return null;
             }
         };
-
         credentialsProvider.setCredentials(new AuthScope(null, -1, null), useJaasCreds);
-        context.setCredentialsProvider(credentialsProvider);
+        client.setCredentialsProvider(credentialsProvider);
+        
         String response = null;
         while (response == null) {
             if (url.startsWith("https://")) {
@@ -116,7 +115,7 @@ public class HadoopStatusGetter {
             HttpGet httpget = new HttpGet(url);
             httpget.addHeader("accept", "application/json");
             try {
-                CloseableHttpResponse httpResponse = client.execute(httpget, context);
+                HttpResponse httpResponse = client.execute(httpget);
                 String redirect = null;
                 org.apache.http.Header h = httpResponse.getFirstHeader("Location");
                 if (h != null) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/b6cdc0d1/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java
index 8890575..2d06a92 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinClient.java
@@ -20,6 +20,7 @@ package org.apache.kylin.jdbc;
 
 import java.io.IOException;
 import java.math.BigDecimal;
+import java.security.cert.CertificateException;
 import java.security.cert.X509Certificate;
 import java.sql.Date;
 import java.sql.Time;
@@ -31,7 +32,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import javax.net.ssl.SSLContext;
 import javax.xml.bind.DatatypeConverter;
 
 import org.apache.calcite.avatica.AvaticaParameter;
@@ -39,17 +39,15 @@ import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.calcite.avatica.ColumnMetaData.ScalarType;
 import org.apache.http.HttpResponse;
-import org.apache.http.client.methods.CloseableHttpResponse;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.client.methods.HttpRequestBase;
-import org.apache.http.conn.ssl.NoopHostnameVerifier;
+import org.apache.http.conn.scheme.Scheme;
+import org.apache.http.conn.ssl.SSLSocketFactory;
 import org.apache.http.conn.ssl.TrustStrategy;
 import org.apache.http.entity.ContentType;
 import org.apache.http.entity.StringEntity;
-import org.apache.http.impl.client.CloseableHttpClient;
-import org.apache.http.impl.client.HttpClients;
-import org.apache.http.ssl.SSLContexts;
+import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.http.util.EntityUtils;
 import org.apache.kylin.jdbc.KylinMeta.KMetaCatalog;
 import org.apache.kylin.jdbc.KylinMeta.KMetaColumn;
@@ -74,25 +72,25 @@ public class KylinClient implements IRemoteClient {
 
     private final KylinConnection conn;
     private final Properties connProps;
-    private CloseableHttpClient httpClient;
+    private DefaultHttpClient httpClient;
     private final ObjectMapper jsonMapper;
 
     public KylinClient(KylinConnection conn) {
         this.conn = conn;
         this.connProps = conn.getConnectionProperties();
-        this.httpClient = HttpClients.createDefault();
+        this.httpClient = new DefaultHttpClient();
         this.jsonMapper = new ObjectMapper();
 
         // trust all certificates
         if (isSSL()) {
             try {
-                TrustStrategy acceptingTrustStrategy = new TrustStrategy() {
-                    public boolean isTrusted(X509Certificate[] certificate, String type) {
+                SSLSocketFactory sslsf = new SSLSocketFactory(new TrustStrategy() {
+                    public boolean isTrusted(final X509Certificate[] chain, String authType) throws CertificateException {
+                        // Oh, I am easy...
                         return true;
                     }
-                };
-                SSLContext sslContext = SSLContexts.custom().loadTrustMaterial(null, acceptingTrustStrategy).build();
-                httpClient = HttpClients.custom().setSSLHostnameVerifier(new NoopHostnameVerifier()).setSSLContext(sslContext).build();
+                });
+                httpClient.getConnectionManager().getSchemeRegistry().register(new Scheme("https", 443, sslsf));
             } catch (Exception e) {
                 throw new RuntimeException("Initialize HTTPS client failed", e);
             }
@@ -227,12 +225,15 @@ public class KylinClient implements IRemoteClient {
         StringEntity requestEntity = new StringEntity("{}", ContentType.create("application/json", "UTF-8"));
         post.setEntity(requestEntity);
 
-        CloseableHttpResponse response = httpClient.execute(post);
+        try {
+            HttpResponse response = httpClient.execute(post);
 
-        if (response.getStatusLine().getStatusCode() != 200 && response.getStatusLine().getStatusCode() != 201) {
-            throw asIOException(post, response);
+            if (response.getStatusLine().getStatusCode() != 200 && response.getStatusLine().getStatusCode() != 201) {
+                throw asIOException(post, response);
+            }
+        } finally {
+            post.releaseConnection();
         }
-        response.close();
     }
 
     @Override
@@ -243,7 +244,7 @@ public class KylinClient implements IRemoteClient {
         HttpGet get = new HttpGet(url);
         addHttpHeaders(get);
 
-        CloseableHttpResponse response = httpClient.execute(get);
+        HttpResponse response = httpClient.execute(get);
 
         if (response.getStatusLine().getStatusCode() != 200 && response.getStatusLine().getStatusCode() != 201) {
             throw asIOException(get, response);
@@ -255,7 +256,7 @@ public class KylinClient implements IRemoteClient {
         List<KMetaTable> tables = convertMetaTables(tableMetaStubs);
         List<KMetaSchema> schemas = convertMetaSchemas(tables);
         List<KMetaCatalog> catalogs = convertMetaCatalogs(schemas);
-        response.close();
+        get.releaseConnection();
         return new KMetaProject(project, catalogs);
     }
 
@@ -368,14 +369,14 @@ public class KylinClient implements IRemoteClient {
         StringEntity requestEntity = new StringEntity(postBody, ContentType.create("application/json", "UTF-8"));
         post.setEntity(requestEntity);
 
-        CloseableHttpResponse response = httpClient.execute(post);
+        HttpResponse response = httpClient.execute(post);
 
         if (response.getStatusLine().getStatusCode() != 200 && response.getStatusLine().getStatusCode() != 201) {
             throw asIOException(post, response);
         }
 
         SQLResponseStub stub = jsonMapper.readValue(response.getEntity().getContent(), SQLResponseStub.class);
-        response.close();
+        post.releaseConnection();
         return stub;
     }
 
@@ -416,8 +417,5 @@ public class KylinClient implements IRemoteClient {
 
     @Override
     public void close() throws IOException {
-        if (httpClient != null) {
-            httpClient.close();
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/b6cdc0d1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1abc4eb..ecb2724 100644
--- a/pom.xml
+++ b/pom.xml
@@ -96,7 +96,7 @@
         <extendedset.version>1.3.4</extendedset.version>
         <kryo.version>4.0.0</kryo.version>
         <ehcache.version>2.10.2.2.21</ehcache.version>
-        <apache-httpclient.version>4.5.2</apache-httpclient.version>
+        <apache-httpclient.version>4.2.5</apache-httpclient.version>
         <roaring.version>0.6.18</roaring.version>
         <cglib.version>3.2.4</cglib.version>
         <supercsv.version>2.4.0</supercsv.version>