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

[01/28] kylin git commit: KYLIN-2033 Broadcaster stronger sync locking and more comments [Forced Update!]

Repository: kylin
Updated Branches:
  refs/heads/KYLIN-1726-2 e75e95285 -> 859230d78 (forced update)


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/KYLIN-1726-2
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);
     }
 
     /**


[16/28] kylin git commit: Revert "Revert "change to upper case""

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

This reverts commit d1e979b4995597ae2ac3fbeb88ba1902d7296782.


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

Branch: refs/heads/KYLIN-1726-2
Commit: de2f4e2a395af57a42aba43f7233bd1bbdd0f6d0
Parents: 8136380
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:57:22 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 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/de2f4e2a/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/de2f4e2a/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
+}


[24/28] kylin git commit: Revert "Revert "KYLIN-1726 Scalable streaming cubing""

Posted by sh...@apache.org.
Revert "Revert "KYLIN-1726 Scalable streaming cubing""

This reverts commit 506cd783132023a06f1669ad248b74bf9d96d0e1.


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

Branch: refs/heads/KYLIN-1726-2
Commit: 8431af45528abb2d39a69b5e762712983573e5a6
Parents: a00d1e3
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:55:59 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/job/DeployUtil.java   |  35 ++-
 .../kylin/job/streaming/Kafka10DataLoader.java  |  80 +++++++
 .../apache/kylin/common/KylinConfigBase.java    |   1 +
 .../java/org/apache/kylin/cube/CubeSegment.java |   1 +
 .../java/org/apache/kylin/cube/ISegment.java    |  39 ----
 .../cube/gridtable/SegmentGTStartAndEnd.java    |   2 +-
 .../cube/model/CubeJoinedFlatTableDesc.java     |   6 +
 .../cube/model/CubeJoinedFlatTableEnrich.java   |   6 +
 .../apache/kylin/gridtable/ScannerWorker.java   |   2 +-
 .../metadata/model/IJoinedFlatTableDesc.java    |   2 +
 .../apache/kylin/metadata/model/ISegment.java   |  36 +++
 .../kylin/engine/mr/BatchMergeJobBuilder2.java  |   3 +
 .../org/apache/kylin/engine/mr/IMRInput.java    |  10 +
 .../java/org/apache/kylin/engine/mr/MRUtil.java |   4 +
 .../test_streaming_table_model_desc.json        |   6 +-
 .../kylin/provision/BuildCubeWithStream.java    | 218 +++++++++++++-----
 .../org/apache/kylin/provision/MockKafka.java   | 191 ++++++++++++++++
 .../apache/kylin/provision/NetworkUtils.java    |  52 +++++
 pom.xml                                         |   2 +-
 .../apache/kylin/source/hive/HiveMRInput.java   |  11 +
 source-kafka/pom.xml                            |  13 +-
 .../apache/kylin/source/kafka/KafkaMRInput.java | 221 +++++++++++++++++++
 .../apache/kylin/source/kafka/KafkaSource.java  |  57 +++++
 .../kylin/source/kafka/KafkaStreamingInput.java |  17 +-
 .../kylin/source/kafka/MergeOffsetStep.java     |  89 ++++++++
 .../kylin/source/kafka/SeekOffsetStep.java      | 119 ++++++++++
 .../kylin/source/kafka/UpdateTimeRangeStep.java | 108 +++++++++
 .../source/kafka/config/KafkaClusterConfig.java |   3 +-
 .../source/kafka/hadoop/KafkaFlatTableJob.java  | 165 ++++++++++++++
 .../kafka/hadoop/KafkaFlatTableMapper.java      |  51 +++++
 .../source/kafka/hadoop/KafkaInputFormat.java   |  98 ++++++++
 .../kafka/hadoop/KafkaInputRecordReader.java    | 166 ++++++++++++++
 .../source/kafka/hadoop/KafkaInputSplit.java    | 102 +++++++++
 .../kylin/source/kafka/util/KafkaClient.java    | 115 ++++++++++
 .../source/kafka/util/KafkaOffsetMapping.java   |  97 ++++++++
 .../kylin/source/kafka/util/KafkaRequester.java |   7 +-
 .../kylin/source/kafka/util/KafkaUtils.java     |   3 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java     |   2 +-
 .../storage/hbase/cube/v2/CubeHBaseRPC.java     |   2 +-
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |   2 +-
 40 files changed, 2024 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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 8c64f91..9b282e3 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -143,14 +143,12 @@ public class DeployUtil {
         deployHiveTables();
     }
 
-    public static void prepareTestDataForStreamingCube(long startTime, long endTime, String cubeName, StreamDataLoader streamDataLoader) throws IOException {
+    public static void prepareTestDataForStreamingCube(long startTime, long endTime, int numberOfRecords, String cubeName, StreamDataLoader streamDataLoader) throws IOException {
         CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
-        List<String> data = StreamingTableDataGenerator.generate(10000, startTime, endTime, cubeInstance.getFactTable());
-        List<String> data2 = StreamingTableDataGenerator.generate(10, endTime, endTime + 300000, cubeInstance.getFactTable());
+        List<String> data = StreamingTableDataGenerator.generate(numberOfRecords, startTime, endTime, cubeInstance.getFactTable());
         TableDesc tableDesc = cubeInstance.getFactTableDesc();
         //load into kafka
         streamDataLoader.loadIntoKafka(data);
-        streamDataLoader.loadIntoKafka(data2);
         logger.info("Write {} messages into {}", data.size(), streamDataLoader.toString());
 
         //csv data for H2 use
@@ -165,7 +163,7 @@ public class DeployUtil {
             sb.append(StringUtils.join(rowColumns, ","));
             sb.append(System.getProperty("line.separator"));
         }
-        overrideFactTableData(sb.toString(), cubeInstance.getFactTable());
+        appendFactTableData(sb.toString(), cubeInstance.getFactTable());
     }
 
     public static void overrideFactTableData(String factTableContent, String factTableName) throws IOException {
@@ -179,6 +177,33 @@ public class DeployUtil {
         in.close();
     }
 
+    public static void appendFactTableData(String factTableContent, String factTableName) throws IOException {
+        // Write to resource store
+        ResourceStore store = ResourceStore.getStore(config());
+
+        InputStream in = new ByteArrayInputStream(factTableContent.getBytes("UTF-8"));
+        String factTablePath = "/data/" + factTableName + ".csv";
+
+        File tmpFile = File.createTempFile(factTableName, "csv");
+        FileOutputStream out = new FileOutputStream(tmpFile);
+
+        try {
+            if (store.exists(factTablePath)) {
+                InputStream oldContent = store.getResource(factTablePath).inputStream;
+                IOUtils.copy(oldContent, out);
+            }
+            IOUtils.copy(in, out);
+            IOUtils.closeQuietly(in);
+
+            store.deleteResource(factTablePath);
+            in = new FileInputStream(tmpFile);
+            store.putResource(factTablePath, in, System.currentTimeMillis());
+        } finally {
+            IOUtils.closeQuietly(out);
+            IOUtils.closeQuietly(in);
+        }
+
+    }
 
     private static void deployHiveTables() throws Exception {
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java b/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
new file mode 100644
index 0000000..a5132af
--- /dev/null
+++ b/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
@@ -0,0 +1,80 @@
+/*
+ * 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.streaming;
+
+import java.util.List;
+import java.util.Properties;
+
+import javax.annotation.Nullable;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kylin.source.kafka.config.BrokerConfig;
+import org.apache.kylin.source.kafka.config.KafkaClusterConfig;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+
+import org.apache.kylin.source.kafka.util.KafkaClient;
+
+/**
+ * Load prepared data into kafka(for test use)
+ */
+public class Kafka10DataLoader extends StreamDataLoader {
+    private static final Logger logger = LoggerFactory.getLogger(Kafka10DataLoader.class);
+    List<KafkaClusterConfig> kafkaClusterConfigs;
+
+    public Kafka10DataLoader(KafkaConfig kafkaConfig) {
+        super(kafkaConfig);
+        this.kafkaClusterConfigs = kafkaConfig.getKafkaClusterConfigs();
+    }
+
+    public void loadIntoKafka(List<String> messages) {
+
+        KafkaClusterConfig clusterConfig = kafkaClusterConfigs.get(0);
+        String brokerList = StringUtils.join(Collections2.transform(clusterConfig.getBrokerConfigs(), new Function<BrokerConfig, String>() {
+            @Nullable
+            @Override
+            public String apply(BrokerConfig brokerConfig) {
+                return brokerConfig.getHost() + ":" + brokerConfig.getPort();
+            }
+        }), ",");
+
+        Properties props = new Properties();
+        props.put("acks", "1");
+        props.put("retry.backoff.ms", "1000");
+        KafkaProducer producer = KafkaClient.getKafkaProducer(brokerList, props);
+
+        int boundary = messages.size() / 10;
+        for (int i = 0; i < messages.size(); ++i) {
+            ProducerRecord<String, String> keyedMessage = new ProducerRecord<String, String>(clusterConfig.getTopic(), String.valueOf(i), messages.get(i));
+            producer.send(keyedMessage);
+            if (i % boundary == 0) {
+                logger.info("sending " + i + " messages to " + this.toString());
+            }
+        }
+        logger.info("sent " + messages.size() + " messages to " + this.toString());
+        producer.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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 ceb188e..914f726 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
@@ -713,6 +713,7 @@ abstract public class KylinConfigBase implements Serializable {
         Map<Integer, String> r = convertKeyToInteger(getPropertiesByPrefix("kylin.source.engine."));
         // ref constants in ISourceAware
         r.put(0, "org.apache.kylin.source.hive.HiveSource");
+        r.put(1, "org.apache.kylin.source.kafka.KafkaSource");
         return r;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index 79397c3..afb0d28 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -37,6 +37,7 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.IBuildable;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.IRealization;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java b/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java
deleted file mode 100644
index 2e1f214..0000000
--- a/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java
+++ /dev/null
@@ -1,39 +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.cube;
-
-import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
-public interface ISegment {
-
-    public String getName();
-
-    public long getDateRangeStart();
-
-    public long getDateRangeEnd();
-
-    public long getSourceOffsetStart();
-
-    public long getSourceOffsetEnd();
-    
-    public DataModelDesc getModel();
-
-    public SegmentStatusEnum getStatus();
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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 21e01b9..b4a82d4 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
@@ -24,7 +24,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.DateFormat;
 import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.cube.ISegment;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.dimension.AbstractDateDimEnc;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.metadata.datatype.DataType;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
index 6aeb617..6ca89c8 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
@@ -26,6 +26,7 @@ import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -162,4 +163,9 @@ public class CubeJoinedFlatTableDesc implements IJoinedFlatTableDesc {
         return cubeDesc.getDistributedByColumn();
     }
 
+    @Override
+    public ISegment getSegment() {
+        return cubeSegment;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java
index 5212859..8af2297 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java
@@ -25,6 +25,7 @@ import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 
@@ -137,4 +138,9 @@ public class CubeJoinedFlatTableEnrich implements IJoinedFlatTableDesc {
         return flatDesc.getDistributedBy();
     }
 
+    @Override
+    public ISegment getSegment() {
+        return flatDesc.getSegment();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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 a0472e5..f26d993 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
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.util.Iterator;
 
-import org.apache.kylin.cube.ISegment;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/core-metadata/src/main/java/org/apache/kylin/metadata/model/IJoinedFlatTableDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IJoinedFlatTableDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IJoinedFlatTableDesc.java
index f3a4107..ffa2680 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/IJoinedFlatTableDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/IJoinedFlatTableDesc.java
@@ -37,4 +37,6 @@ public interface IJoinedFlatTableDesc {
     long getSourceOffsetEnd();
     
     TblColRef getDistributedBy();
+
+    ISegment getSegment();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISegment.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISegment.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISegment.java
new file mode 100644
index 0000000..f69ae3f
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISegment.java
@@ -0,0 +1,36 @@
+/*
+ * 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.model;
+
+public interface ISegment {
+
+    public String getName();
+
+    public long getDateRangeStart();
+
+    public long getDateRangeEnd();
+
+    public long getSourceOffsetStart();
+
+    public long getSourceOffsetEnd();
+    
+    public DataModelDesc getModel();
+
+    public SegmentStatusEnum getStatus();
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
index 129d525..badf628 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchMergeJobBuilder2.java
@@ -34,10 +34,12 @@ public class BatchMergeJobBuilder2 extends JobBuilderSupport {
     private static final Logger logger = LoggerFactory.getLogger(BatchMergeJobBuilder2.class);
 
     private final IMROutput2.IMRBatchMergeOutputSide2 outputSide;
+    private final IMRInput.IMRBatchMergeInputSide inputSide;
 
     public BatchMergeJobBuilder2(CubeSegment mergeSegment, String submitter) {
         super(mergeSegment, submitter);
         this.outputSide = MRUtil.getBatchMergeOutputSide2(seg);
+        this.inputSide = MRUtil.getBatchMergeInputSide(seg);
     }
 
     public CubingJob build() {
@@ -55,6 +57,7 @@ public class BatchMergeJobBuilder2 extends JobBuilderSupport {
         }
 
         // Phase 1: Merge Dictionary
+        inputSide.addStepPhase1_MergeDictionary(result);
         result.addTask(createMergeDictionaryStep(mergingSegmentIds));
         result.addTask(createMergeStatisticsStep(cubeSegment, mergingSegmentIds, getStatisticsPath(jobId)));
         outputSide.addStepPhase1_MergeDictionary(result);

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
index 582052f..62cede9 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
@@ -21,6 +21,7 @@ package org.apache.kylin.engine.mr;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.kylin.job.execution.DefaultChainedExecutable;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.metadata.model.TableDesc;
 
 /**
@@ -34,6 +35,9 @@ public interface IMRInput {
     /** Return an InputFormat that reads from specified table. */
     public IMRTableInputFormat getTableInputFormat(TableDesc table);
 
+    /** Return a helper to participate in batch cubing merge job flow. */
+    public IMRBatchMergeInputSide getBatchMergeInputSide(ISegment seg);
+
     /**
      * Utility that configures mapper to read from a table.
      */
@@ -67,4 +71,10 @@ public interface IMRInput {
         public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow);
     }
 
+    public interface IMRBatchMergeInputSide {
+
+        /** Add step that executes before merge dictionary and before merge cube. */
+        public void addStepPhase1_MergeDictionary(DefaultChainedExecutable jobFlow);
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
index 2c3b77f..67eef5e 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
@@ -71,6 +71,10 @@ public class MRUtil {
         return StorageFactory.createEngineAdapter(seg, IMROutput2.class).getBatchMergeOutputSide(seg);
     }
 
+    public static IMRInput.IMRBatchMergeInputSide getBatchMergeInputSide(CubeSegment seg) {
+        return SourceFactory.createEngineAdapter(seg, IMRInput.class).getBatchMergeInputSide(seg);
+    }
+
     // use this method instead of ToolRunner.run() because ToolRunner.run() is not thread-sale
     // Refer to: http://stackoverflow.com/questions/22462665/is-hadoops-toorunner-thread-safe
     public static int runMRJob(Tool tool, String[] args) throws Exception {

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json b/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json
index cfb889a..e6977e1 100644
--- a/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json
+++ b/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json
@@ -4,7 +4,7 @@
   "name": "test_streaming_table_model_desc",
   "dimensions": [
     {
-      "table": "default.streaming_table",
+      "table": "DEFAULT.STREAMING_TABLE",
       "columns": [
         "minute_start",
         "hour_start",
@@ -20,10 +20,10 @@
     "item_count"
   ],
   "last_modified": 0,
-  "fact_table": "default.streaming_table",
+  "fact_table": "DEFAULT.STREAMING_TABLE",
   "filter_condition": null,
   "partition_desc": {
-    "partition_date_column": "default.streaming_table.minute_start",
+    "partition_date_column": "DEFAULT.STREAMING_TABLE.minute_start",
     "partition_date_start": 0,
     "partition_type": "APPEND"
   }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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 9490560..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
@@ -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.
@@ -20,24 +20,36 @@ package org.apache.kylin.provision;
 
 import java.io.File;
 import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.TimeZone;
 import java.util.UUID;
 
+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.DateFormat;
 import org.apache.kylin.common.util.HBaseMetadataTestCase;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.engine.streaming.OneOffStreamingBuilder;
+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.streaming.KafkaDataLoader;
-import org.apache.kylin.metadata.realization.RealizationType;
+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.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.StorageCleanupJob;
+import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
+import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,31 +58,123 @@ import org.slf4j.LoggerFactory;
  */
 public class BuildCubeWithStream {
 
-    private static final Logger logger = LoggerFactory.getLogger(BuildCubeWithStream.class);
-    private static final String cubeName = "test_streaming_table_cube";
-    private static final long startTime = DateFormat.stringToMillis("2015-01-01 00:00:00");
-    private static final long endTime = DateFormat.stringToMillis("2015-01-03 00:00:00");
-    private static final long batchInterval = 16 * 60 * 60 * 1000;//16 hours
+    private static final Logger logger = LoggerFactory.getLogger(org.apache.kylin.provision.BuildCubeWithStream.class);
 
-    private KylinConfig kylinConfig;
+    private CubeManager cubeManager;
+    private DefaultScheduler scheduler;
+    protected ExecutableManager jobService;
+    private static final String cubeName = "test_streaming_table_cube";
 
-    public static void main(String[] args) throws Exception {
+    private KafkaConfig kafkaConfig;
+    private MockKafka kafkaServer;
 
-        try {
-            beforeClass();
+    public void before() throws Exception {
+        deployEnv();
 
-            BuildCubeWithStream buildCubeWithStream = new BuildCubeWithStream();
-            buildCubeWithStream.before();
-            buildCubeWithStream.build();
-            logger.info("Build is done");
-            buildCubeWithStream.cleanup();
-            logger.info("Going to exit");
-            System.exit(0);
-        } catch (Exception e) {
-            logger.error("error", e);
-            System.exit(1);
+        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 {
+        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");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+        long date1 = 0;
+        long date2 = f.parse("2013-01-01").getTime();
+
+        int numberOfRecrods1 = 10000;
+        generateStreamData(date1, date2, numberOfRecrods1);
+        buildSegment(cubeName, 0, Long.MAX_VALUE);
+
+        long date3 = f.parse("2013-04-01").getTime();
+        int numberOfRecrods2 = 5000;
+        generateStreamData(date2, date3, numberOfRecrods2);
+        buildSegment(cubeName, 0, Long.MAX_VALUE);
+
+        //merge
+        mergeSegment(cubeName, 0, 15000);
+
+    }
+
+    private String mergeSegment(String cubeName, long startOffset, long endOffset) throws Exception {
+        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) 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());
+        return job.getId();
+    }
 
+    private String 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.getId();
+    }
+
+    protected void deployEnv() throws IOException {
+        DeployUtil.overrideJobJarLocations();
+        //DeployUtil.initCliWorkDir();
+        //DeployUtil.deployMetadata();
     }
 
     public static void beforeClass() throws Exception {
@@ -83,44 +187,54 @@ public class BuildCubeWithStream {
         HBaseMetadataTestCase.staticCreateTestMetadata(HBaseMetadataTestCase.SANDBOX_TEST_DATA);
     }
 
-    protected void deployEnv() throws IOException {
-        DeployUtil.overrideJobJarLocations();
+    public static void afterClass() throws Exception {
+        HBaseMetadataTestCase.staticCleanupTestMetadata();
     }
 
-    public void before() throws Exception {
-        deployEnv();
+    public void after() {
+        kafkaServer.stop();
+    }
 
-        kylinConfig = KylinConfig.getInstanceFromEnv();
-        final CubeInstance cubeInstance = CubeManager.getInstance(kylinConfig).getCube(cubeName);
-        final String factTable = cubeInstance.getFactTable();
-        final StreamingConfig config = StreamingManager.getInstance(kylinConfig).getStreamingConfig(factTable);
+    protected void waitForJob(String jobId) {
+        while (true) {
+            AbstractExecutable job = jobService.getJob(jobId);
+            if (job.getStatus() == ExecutableState.SUCCEED || job.getStatus() == ExecutableState.ERROR) {
+                break;
+            } else {
+                try {
+                    Thread.sleep(5000);
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
 
-        //Use a random topic for kafka data stream
-        KafkaConfig streamingConfig = KafkaConfigManager.getInstance(kylinConfig).getKafkaConfig(config.getName());
-        streamingConfig.setTopic(UUID.randomUUID().toString());
-        KafkaConfigManager.getInstance(kylinConfig).saveKafkaConfig(streamingConfig);
+    public static void main(String[] args) throws Exception {
+        try {
+            beforeClass();
 
-        DeployUtil.prepareTestDataForStreamingCube(startTime, endTime, cubeName, new KafkaDataLoader(streamingConfig));
-    }
+            BuildCubeWithStream buildCubeWithStream = new BuildCubeWithStream();
+            buildCubeWithStream.before();
+            buildCubeWithStream.build();
+            logger.info("Build is done");
+            buildCubeWithStream.after();
+            afterClass();
+            logger.info("Going to exit");
+            System.exit(0);
+        } catch (Exception e) {
+            logger.error("error", e);
+            System.exit(1);
+        }
 
-    public void cleanup() throws Exception {
-        cleanupOldStorage();
-        HBaseMetadataTestCase.staticCleanupTestMetadata();
     }
 
     protected int cleanupOldStorage() throws Exception {
         String[] args = { "--delete", "true" };
 
-        StorageCleanupJob cli = new StorageCleanupJob();
-        cli.execute(args);
+        //        KapStorageCleanupCLI cli = new KapStorageCleanupCLI();
+        //        cli.execute(args);
         return 0;
     }
 
-    public void build() throws Exception {
-        logger.info("start time:" + startTime + " end time:" + endTime + " batch interval:" + batchInterval + " batch count:" + ((endTime - startTime) / batchInterval));
-        for (long start = startTime; start < endTime; start += batchInterval) {
-            logger.info(String.format("build batch:{%d, %d}", start, start + batchInterval));
-            new OneOffStreamingBuilder(RealizationType.CUBE, cubeName, start, start + batchInterval).build().run();
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/kylin-it/src/test/java/org/apache/kylin/provision/MockKafka.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/MockKafka.java b/kylin-it/src/test/java/org/apache/kylin/provision/MockKafka.java
new file mode 100644
index 0000000..3f47923
--- /dev/null
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/MockKafka.java
@@ -0,0 +1,191 @@
+/*
+ * 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 java.io.UnsupportedEncodingException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Random;
+import java.util.UUID;
+
+import org.I0Itec.zkclient.ZkClient;
+import org.I0Itec.zkclient.ZkConnection;
+import org.I0Itec.zkclient.exception.ZkMarshallingError;
+import org.I0Itec.zkclient.serialize.ZkSerializer;
+import org.apache.kafka.common.requests.MetadataResponse;
+
+import kafka.admin.AdminUtils;
+import kafka.server.KafkaConfig;
+import kafka.server.KafkaServerStartable;
+import kafka.utils.ZkUtils;
+
+public class MockKafka {
+    private static Properties createProperties(ZkConnection zkServerConnection, String logDir, String port, String brokerId) {
+        Properties properties = new Properties();
+        properties.put("port", port);
+        properties.put("broker.id", brokerId);
+        properties.put("log.dirs", logDir);
+        properties.put("host.name", "localhost");
+        properties.put("offsets.topic.replication.factor", "1");
+        properties.put("delete.topic.enable", "true");
+        properties.put("zookeeper.connect", zkServerConnection.getServers());
+        String ip = NetworkUtils.getLocalIp();
+        properties.put("listeners", "PLAINTEXT://" + ip + ":" + port);
+        properties.put("advertised.listeners", "PLAINTEXT://" + ip + ":" + port);
+        return properties;
+    }
+
+    private KafkaServerStartable kafkaServer;
+
+    private ZkConnection zkConnection;
+
+    public MockKafka(ZkConnection zkServerConnection) {
+        this(zkServerConnection, System.getProperty("java.io.tmpdir") + "/" + UUID.randomUUID().toString(), "9092", "1");
+        start();
+    }
+
+    private MockKafka(Properties properties) {
+        KafkaConfig kafkaConfig = new KafkaConfig(properties);
+        kafkaServer = new KafkaServerStartable(kafkaConfig);
+    }
+
+    public MockKafka(ZkConnection zkServerConnection, int port, int brokerId) {
+        this(zkServerConnection, System.getProperty("java.io.tmpdir") + "/" + UUID.randomUUID().toString(), String.valueOf(port), String.valueOf(brokerId));
+        start();
+    }
+
+    private MockKafka(ZkConnection zkServerConnection, String logDir, String port, String brokerId) {
+        this(createProperties(zkServerConnection, logDir, port, brokerId));
+        this.zkConnection = zkServerConnection;
+        System.out.println(String.format("Kafka %s:%s dir:%s", kafkaServer.serverConfig().brokerId(), kafkaServer.serverConfig().port(), kafkaServer.serverConfig().logDirs()));
+    }
+
+    public void createTopic(String topic, int partition, int replication) {
+        ZkClient zkClient = new ZkClient(zkConnection);
+        ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false);
+        zkClient.setZkSerializer(new ZKStringSerializer());
+        AdminUtils.createTopic(zkUtils, topic, partition, replication, new Properties(), null);
+        zkClient.close();
+    }
+
+    public void createTopic(String topic) {
+        this.createTopic(topic, 1, 1);
+    }
+
+    public MetadataResponse.TopicMetadata fetchTopicMeta(String topic) {
+        ZkClient zkClient = new ZkClient(zkConnection);
+        ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false);
+        zkClient.setZkSerializer(new ZKStringSerializer());
+        MetadataResponse.TopicMetadata topicMetadata = AdminUtils.fetchTopicMetadataFromZk(topic, zkUtils);
+        zkClient.close();
+        return topicMetadata;
+    }
+
+    /**
+     * Delete may not work
+     *
+     * @param topic
+     */
+    public void deleteTopic(String topic) {
+        ZkClient zkClient = new ZkClient(zkConnection);
+        ZkUtils zkUtils = new ZkUtils(zkClient, zkConnection, false);
+        zkClient.setZkSerializer(new ZKStringSerializer());
+        AdminUtils.deleteTopic(zkUtils, topic);
+        zkClient.close();
+    }
+
+    public String getConnectionString() {
+        return String.format("%s:%d", kafkaServer.serverConfig().hostName(), kafkaServer.serverConfig().port());
+    }
+
+    public void start() {
+        kafkaServer.startup();
+        System.out.println("embedded kafka is up");
+    }
+
+    public void stop() {
+        kafkaServer.shutdown();
+        System.out.println("embedded kafka down");
+    }
+
+    public MetadataResponse.TopicMetadata waitTopicUntilReady(String topic) {
+        boolean isReady = false;
+        MetadataResponse.TopicMetadata topicMeta = null;
+        while (!isReady) {
+            Random random = new Random();
+            topicMeta = this.fetchTopicMeta(topic);
+            List<MetadataResponse.PartitionMetadata> partitionsMetadata = topicMeta.partitionMetadata();
+            Iterator<MetadataResponse.PartitionMetadata> iterator = partitionsMetadata.iterator();
+            boolean hasGotLeader = true;
+            boolean hasGotReplica = true;
+            while (iterator.hasNext()) {
+                MetadataResponse.PartitionMetadata partitionMeta = iterator.next();
+                hasGotLeader &= (!partitionMeta.leader().isEmpty());
+                if (partitionMeta.leader().isEmpty()) {
+                    System.out.println("Partition leader is not ready, wait 1s.");
+                    break;
+                }
+                hasGotReplica &= (!partitionMeta.replicas().isEmpty());
+                if (partitionMeta.replicas().isEmpty()) {
+                    System.out.println("Partition replica is not ready, wait 1s.");
+                    break;
+                }
+            }
+            isReady = hasGotLeader & hasGotReplica;
+            if (!isReady) {
+                try {
+                    Thread.sleep(1000);
+                } catch (InterruptedException e) {
+                }
+            }
+        }
+        return topicMeta;
+    }
+
+    public String getZookeeperConnection() {
+        return this.zkConnection.getServers();
+    }
+}
+
+class ZKStringSerializer implements ZkSerializer {
+
+    @Override
+    public byte[] serialize(Object data) throws ZkMarshallingError {
+        byte[] bytes = null;
+        try {
+            bytes = data.toString().getBytes("UTF-8");
+        } catch (UnsupportedEncodingException e) {
+            throw new ZkMarshallingError(e);
+        }
+        return bytes;
+    }
+
+    @Override
+    public Object deserialize(byte[] bytes) throws ZkMarshallingError {
+        if (bytes == null)
+            return null;
+        else
+            try {
+                return new String(bytes, "UTF-8");
+            } catch (UnsupportedEncodingException e) {
+                throw new ZkMarshallingError(e);
+            }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/kylin-it/src/test/java/org/apache/kylin/provision/NetworkUtils.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/NetworkUtils.java b/kylin-it/src/test/java/org/apache/kylin/provision/NetworkUtils.java
new file mode 100644
index 0000000..98f6d04
--- /dev/null
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/NetworkUtils.java
@@ -0,0 +1,52 @@
+/*
+ * 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 java.net.Inet4Address;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.util.Enumeration;
+
+public class NetworkUtils {
+
+    public static String getLocalIp() {
+        try {
+            Enumeration<NetworkInterface> interfaces = NetworkInterface.getNetworkInterfaces();
+            while (interfaces.hasMoreElements()) {
+                NetworkInterface iface = interfaces.nextElement();
+                if (iface.isLoopback() || !iface.isUp() || iface.isVirtual() || iface.isPointToPoint())
+                    continue;
+                if (iface.getName().startsWith("vboxnet"))
+                    continue;
+
+                Enumeration<InetAddress> addresses = iface.getInetAddresses();
+                while (addresses.hasMoreElements()) {
+                    InetAddress addr = addresses.nextElement();
+                    final String ip = addr.getHostAddress();
+                    if (Inet4Address.class == addr.getClass())
+                        return ip;
+                }
+            }
+        } catch (SocketException e) {
+            throw new RuntimeException(e);
+        }
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1fdf81a..09ef0e8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -55,7 +55,7 @@
 
         <!-- HBase versions -->
         <hbase-hadoop2.version>0.98.8-hadoop2</hbase-hadoop2.version>
-        <kafka.version>0.8.1</kafka.version>
+        <kafka.version>0.10.0.0</kafka.version>
 
         <!-- Hadoop deps, keep compatible with hadoop2.version -->
         <zookeeper.version>3.4.6</zookeeper.version>

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index 520d7cc..09ac522 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -50,6 +50,7 @@ import org.apache.kylin.job.execution.ExecutableContext;
 import org.apache.kylin.job.execution.ExecuteResult;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.metadata.model.LookupDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.slf4j.Logger;
@@ -69,6 +70,16 @@ public class HiveMRInput implements IMRInput {
         return new HiveTableInputFormat(table.getIdentity());
     }
 
+    @Override
+    public IMRBatchMergeInputSide getBatchMergeInputSide(ISegment seg) {
+        return new IMRBatchMergeInputSide() {
+            @Override
+            public void addStepPhase1_MergeDictionary(DefaultChainedExecutable jobFlow) {
+                // doing nothing
+            }
+        };
+    }
+
     public static class HiveTableInputFormat implements IMRTableInputFormat {
         final String dbName;
         final String tableName;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/source-kafka/pom.xml b/source-kafka/pom.xml
index 90c2211..212f4c6 100644
--- a/source-kafka/pom.xml
+++ b/source-kafka/pom.xml
@@ -32,10 +32,11 @@
 
     </parent>
 
-    <properties>
-    </properties>
-
     <dependencies>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-engine-mr</artifactId>
+        </dependency>
 
         <dependency>
             <groupId>org.apache.kylin</groupId>
@@ -60,16 +61,10 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
-            <groupId>org.apache.hive.hcatalog</groupId>
-            <artifactId>hive-hcatalog-core</artifactId>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>
         </dependency>
     </dependencies>
 
-
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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
new file mode 100644
index 0000000..cfce137
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
@@ -0,0 +1,221 @@
+/*
+ * 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 com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.apache.kylin.metadata.model.ISegment;
+import org.apache.kylin.source.kafka.hadoop.KafkaFlatTableJob;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.StreamingMessage;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableDesc;
+import org.apache.kylin.engine.mr.IMRInput;
+import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.MapReduceExecutable;
+import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.job.JoinedFlatTable;
+import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+public class KafkaMRInput implements IMRInput {
+
+    CubeSegment cubeSegment;
+
+    @Override
+    public IMRBatchCubingInputSide getBatchCubingInputSide(IJoinedFlatTableDesc flatDesc) {
+        this.cubeSegment = (CubeSegment)flatDesc.getSegment();
+        return new BatchCubingInputSide(cubeSegment);
+    }
+
+    @Override
+    public IMRTableInputFormat getTableInputFormat(TableDesc table) {
+        KafkaConfigManager kafkaConfigManager = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv());
+        KafkaConfig kafkaConfig = kafkaConfigManager.getKafkaConfig(table.getIdentity());
+        List<TblColRef> columns = Lists.transform(Arrays.asList(table.getColumns()), new Function<ColumnDesc, TblColRef>() {
+            @Nullable
+            @Override
+            public TblColRef apply(ColumnDesc input) {
+                return input.getRef();
+            }
+        });
+
+        return new KafkaTableInputFormat(cubeSegment, columns, kafkaConfig, null);
+    }
+
+    @Override
+    public IMRBatchMergeInputSide getBatchMergeInputSide(ISegment seg) {
+        return new KafkaMRBatchMergeInputSide((CubeSegment) seg);
+    }
+
+    public static class KafkaTableInputFormat implements IMRTableInputFormat {
+        private final CubeSegment cubeSegment;
+        private List<TblColRef> columns;
+        private StreamingParser streamingParser;
+        private KafkaConfig kafkaConfig;
+        private final JobEngineConfig conf;
+
+        public KafkaTableInputFormat(CubeSegment cubeSegment, List<TblColRef> columns, KafkaConfig kafkaConfig, JobEngineConfig conf) {
+            this.cubeSegment = cubeSegment;
+            this.columns = columns;
+            this.kafkaConfig = kafkaConfig;
+            this.conf = conf;
+        }
+
+        @Override
+        public void configureJob(Job job) {
+            job.setInputFormatClass(SequenceFileInputFormat.class);
+            job.setMapOutputValueClass(Text.class);
+            String jobId = job.getConfiguration().get(BatchConstants.ARG_CUBING_JOB_ID);
+            IJoinedFlatTableDesc flatHiveTableDesc = new CubeJoinedFlatTableDesc(cubeSegment);
+            String inputPath = JoinedFlatTable.getTableDir(flatHiveTableDesc, JobBuilderSupport.getJobWorkingDir(conf, jobId));
+            try {
+                FileInputFormat.addInputPath(job, new Path(inputPath));
+            } catch (IOException e) {
+                throw new IllegalStateException(e);
+            }
+        }
+
+        @Override
+        public String[] parseMapperInput(Object mapperInput) {
+            if (streamingParser == null) {
+                try {
+                    streamingParser = StreamingParser.getStreamingParser(kafkaConfig.getParserName(), kafkaConfig.getParserProperties(), columns);
+                } catch (ReflectiveOperationException e) {
+                    throw new IllegalArgumentException();
+                }
+            }
+            Text text = (Text) mapperInput;
+            ByteBuffer buffer = ByteBuffer.wrap(text.getBytes(), 0, text.getLength()).slice();
+            StreamingMessage streamingMessage = streamingParser.parse(buffer);
+            return streamingMessage.getData().toArray(new String[streamingMessage.getData().size()]);
+        }
+
+    }
+
+    public static class BatchCubingInputSide implements IMRBatchCubingInputSide {
+
+        final JobEngineConfig conf;
+        final CubeSegment seg;
+        private String outputPath;
+
+        public BatchCubingInputSide(CubeSegment seg) {
+            this.conf = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
+            this.seg = seg;
+        }
+
+        @Override
+        public void addStepPhase1_CreateFlatTable(DefaultChainedExecutable jobFlow) {
+            jobFlow.addTask(createUpdateSegmentOffsetStep(jobFlow.getId()));
+            jobFlow.addTask(createSaveKafkaDataStep(jobFlow.getId()));
+        }
+
+        public SeekOffsetStep createUpdateSegmentOffsetStep(String jobId) {
+            final SeekOffsetStep result = new SeekOffsetStep();
+            result.setName("Seek and update offset step");
+
+            CubingExecutableUtil.setCubeName(seg.getRealization().getName(), result.getParams());
+            CubingExecutableUtil.setSegmentId(seg.getUuid(), result.getParams());
+            CubingExecutableUtil.setCubingJobId(jobId, result.getParams());
+
+            return result;
+        }
+
+        private MapReduceExecutable createSaveKafkaDataStep(String jobId) {
+            MapReduceExecutable result = new MapReduceExecutable();
+
+            IJoinedFlatTableDesc flatHiveTableDesc = new CubeJoinedFlatTableDesc(seg);
+            outputPath = JoinedFlatTable.getTableDir(flatHiveTableDesc, JobBuilderSupport.getJobWorkingDir(conf, jobId));
+            result.setName("Save data from Kafka");
+            result.setMapReduceJobClass(KafkaFlatTableJob.class);
+            JobBuilderSupport jobBuilderSupport = new JobBuilderSupport(seg, "system");
+            StringBuilder cmd = new StringBuilder();
+            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_NAME, seg.getName());
+            JobBuilderSupport.appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_Save_Kafka_Data_" + seg.getRealization().getName() + "_Step");
+
+            result.setMapReduceParams(cmd.toString());
+            return result;
+        }
+
+        @Override
+        public void addStepPhase4_Cleanup(DefaultChainedExecutable jobFlow) {
+            final UpdateTimeRangeStep result = new UpdateTimeRangeStep();
+            result.setName("Update Segment Time Range");
+            CubingExecutableUtil.setCubeName(seg.getRealization().getName(), result.getParams());
+            CubingExecutableUtil.setSegmentId(seg.getUuid(), result.getParams());
+            CubingExecutableUtil.setCubingJobId(jobFlow.getId(), result.getParams());
+            JobBuilderSupport jobBuilderSupport = new JobBuilderSupport(seg, "SYSTEM");
+            result.getParams().put(BatchConstants.CFG_OUTPUT_PATH, jobBuilderSupport.getFactDistinctColumnsPath(jobFlow.getId()));
+            jobFlow.addTask(result);
+
+        }
+
+        @Override
+        public IMRTableInputFormat getFlatTableInputFormat() {
+            KafkaConfigManager kafkaConfigManager = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv());
+            KafkaConfig kafkaConfig = kafkaConfigManager.getKafkaConfig(seg.getRealization().getFactTable());
+            List<TblColRef> columns = new CubeJoinedFlatTableDesc(seg).getAllColumns();
+
+            return new KafkaTableInputFormat(seg, columns, kafkaConfig, conf);
+
+        }
+
+    }
+
+    class KafkaMRBatchMergeInputSide implements IMRBatchMergeInputSide {
+
+        private CubeSegment cubeSegment;
+
+        KafkaMRBatchMergeInputSide(CubeSegment cubeSegment) {
+            this.cubeSegment = cubeSegment;
+        }
+
+        @Override
+        public void addStepPhase1_MergeDictionary(DefaultChainedExecutable jobFlow) {
+
+            final MergeOffsetStep result = new MergeOffsetStep();
+            result.setName("Merge offset step");
+
+            CubingExecutableUtil.setCubeName(cubeSegment.getRealization().getName(), result.getParams());
+            CubingExecutableUtil.setSegmentId(cubeSegment.getUuid(), result.getParams());
+            CubingExecutableUtil.setCubingJobId(jobFlow.getId(), result.getParams());
+            jobFlow.addTask(result);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java
new file mode 100644
index 0000000..d039583
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java
@@ -0,0 +1,57 @@
+/*
+ * 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 com.google.common.collect.Lists;
+import org.apache.kylin.engine.mr.IMRInput;
+import org.apache.kylin.engine.streaming.StreamingConfig;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.source.ISource;
+import org.apache.kylin.source.ReadableTable;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
+
+import java.util.List;
+
+//used by reflection
+public class KafkaSource implements ISource {
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <I> I adaptToBuildEngine(Class<I> engineInterface) {
+        if (engineInterface == IMRInput.class) {
+            return (I) new KafkaMRInput();
+        } else {
+            throw new RuntimeException("Cannot adapt to " + engineInterface);
+        }
+    }
+
+    @Override
+    public ReadableTable createReadableTable(TableDesc tableDesc) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<String> getMRDependentResources(TableDesc table) {
+        List<String> dependentResources = Lists.newArrayList();
+        dependentResources.add(KafkaConfig.concatResourcePath(table.getIdentity()));
+        dependentResources.add(StreamingConfig.concatResourcePath(table.getIdentity()));
+        return dependentResources;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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 66142c5..78a67c2 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
@@ -25,6 +25,9 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
+import com.google.common.base.Function;
+import kafka.cluster.BrokerEndPoint;
+import org.apache.kafka.common.protocol.SecurityProtocol;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.StreamingBatch;
@@ -51,6 +54,8 @@ import kafka.javaapi.FetchResponse;
 import kafka.javaapi.PartitionMetadata;
 import kafka.message.MessageAndOffset;
 
+import javax.annotation.Nullable;
+
 @SuppressWarnings("unused")
 public class KafkaStreamingInput implements IStreamingInput {
 
@@ -136,8 +141,16 @@ public class KafkaStreamingInput implements IStreamingInput {
                 if (partitionMetadata.errorCode() != 0) {
                     logger.warn("PartitionMetadata errorCode: " + partitionMetadata.errorCode());
                 }
-                replicaBrokers = partitionMetadata.replicas();
-                return partitionMetadata.leader();
+                replicaBrokers = Lists.transform(partitionMetadata.replicas(), new Function<BrokerEndPoint, Broker>() {
+                    @Nullable
+                    @Override
+                    public Broker apply(@Nullable BrokerEndPoint brokerEndPoint) {
+                        return new Broker(brokerEndPoint, SecurityProtocol.PLAINTEXT);
+                    }
+                });
+                BrokerEndPoint leaderEndpoint = partitionMetadata.leader();
+
+                return new Broker(leaderEndpoint, SecurityProtocol.PLAINTEXT);
             } else {
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/MergeOffsetStep.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/MergeOffsetStep.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/MergeOffsetStep.java
new file mode 100644
index 0000000..a21b980
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/MergeOffsetStep.java
@@ -0,0 +1,89 @@
+/*
+ * 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.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.Maps;
+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.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kylin.source.kafka.util.KafkaOffsetMapping;
+
+/**
+ */
+public class MergeOffsetStep extends AbstractExecutable {
+
+    private static final Logger logger = LoggerFactory.getLogger(MergeOffsetStep.class);
+    public MergeOffsetStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeManager cubeManager = CubeManager.getInstance(context.getConfig());
+        final CubeInstance cube = cubeManager.getCube(CubingExecutableUtil.getCubeName(this.getParams()));
+        final CubeSegment segment = cube.getSegmentById(CubingExecutableUtil.getSegmentId(this.getParams()));
+
+        List<CubeSegment> mergingSegs = cube.getMergingSegments(segment);
+        Map<Integer, Long> mergedStartOffsets = Maps.newHashMap();
+        Map<Integer, Long> mergedEndOffsets = Maps.newHashMap();
+
+        long dateRangeStart = Long.MAX_VALUE, dateRangeEnd = 0;
+        for (CubeSegment seg: mergingSegs) {
+            Map<Integer, Long> startOffsets = KafkaOffsetMapping.parseOffsetStart(seg);
+            Map<Integer, Long> endOffsets = KafkaOffsetMapping.parseOffsetEnd(seg);
+
+            for (Integer partition : startOffsets.keySet()) {
+                long currentStart = mergedStartOffsets.get(partition) != null ? Long.valueOf(mergedStartOffsets.get(partition)) : Long.MAX_VALUE;
+                long currentEnd = mergedEndOffsets.get(partition) != null ? Long.valueOf(mergedEndOffsets.get(partition)) : 0;
+                mergedStartOffsets.put(partition, Math.min(currentStart, startOffsets.get(partition)));
+                mergedEndOffsets.put(partition, Math.max(currentEnd, endOffsets.get(partition)));
+            }
+            dateRangeStart = Math.min(dateRangeStart, seg.getDateRangeStart());
+            dateRangeEnd = Math.max(dateRangeEnd, seg.getDateRangeEnd());
+        }
+
+        KafkaOffsetMapping.saveOffsetStart(segment, mergedStartOffsets);
+        KafkaOffsetMapping.saveOffsetEnd(segment, mergedEndOffsets);
+        segment.setDateRangeStart(dateRangeStart);
+        segment.setDateRangeEnd(dateRangeEnd);
+
+        CubeUpdate cubeBuilder = new CubeUpdate(cube);
+        cubeBuilder.setToUpdateSegs(segment);
+        try {
+            cubeManager.updateCube(cubeBuilder);
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (IOException e) {
+            logger.error("fail to update cube segment offset", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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
new file mode 100644
index 0000000..5dca93f
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.kylin.source.kafka.util.KafkaClient;
+import org.apache.kylin.source.kafka.util.KafkaOffsetMapping;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.PartitionInfo;
+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.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ */
+public class SeekOffsetStep extends AbstractExecutable {
+
+    private static final Logger logger = LoggerFactory.getLogger(SeekOffsetStep.class);
+
+    public SeekOffsetStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeManager cubeManager = CubeManager.getInstance(context.getConfig());
+        final CubeInstance cube = cubeManager.getCube(CubingExecutableUtil.getCubeName(this.getParams()));
+        final CubeSegment segment = cube.getSegmentById(CubingExecutableUtil.getSegmentId(this.getParams()));
+
+        Map<Integer, Long> startOffsets = KafkaOffsetMapping.parseOffsetStart(segment);
+        Map<Integer, Long> endOffsets = KafkaOffsetMapping.parseOffsetEnd(segment);
+
+        if (startOffsets.size() > 0 && endOffsets.size() > 0 && startOffsets.size() == endOffsets.size()) {
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "skipped, as the offset is provided.");
+        }
+
+        final KafkaConfig kafakaConfig = KafkaConfigManager.getInstance(context.getConfig()).getKafkaConfig(cube.getFactTable());
+        final String brokers = KafkaClient.getKafkaBrokers(kafakaConfig);
+        final String topic = kafakaConfig.getTopic();
+        try (final KafkaConsumer consumer = KafkaClient.getKafkaConsumer(brokers, cube.getName(), null)) {
+            final List<PartitionInfo> partitionInfos = consumer.partitionsFor(topic);
+
+            if (startOffsets.isEmpty()) {
+                // user didn't specify start offset, use the biggest offset in existing segments as start
+                for (CubeSegment seg : cube.getSegments()) {
+                    Map<Integer, Long> segEndOffset = KafkaOffsetMapping.parseOffsetEnd(seg);
+                    for (PartitionInfo partition : partitionInfos) {
+                        int partitionId = partition.partition();
+                        if (segEndOffset.containsKey(partitionId)) {
+                            startOffsets.put(partitionId, Math.max(startOffsets.containsKey(partitionId) ? startOffsets.get(partitionId) : 0, segEndOffset.get(partitionId)));
+                        }
+                    }
+                }
+
+                if (partitionInfos.size() > startOffsets.size()) {
+                    // has new partition added
+                    for (int x = startOffsets.size(); x < partitionInfos.size(); x++) {
+                        long earliest = KafkaClient.getEarliestOffset(consumer, topic, partitionInfos.get(x).partition());
+                        startOffsets.put(partitionInfos.get(x).partition(), earliest);
+                    }
+                }
+
+                logger.info("Get start offset for segment " + segment.getName() + ": " + startOffsets.toString());
+            }
+
+            if (endOffsets.isEmpty()) {
+                // user didn't specify end offset, use latest offset in kafka
+                for (PartitionInfo partitionInfo : partitionInfos) {
+                    long latest = KafkaClient.getLatestOffset(consumer, topic, partitionInfo.partition());
+                    endOffsets.put(partitionInfo.partition(), latest);
+                }
+
+                logger.info("Get end offset for segment " + segment.getName() + ": " + endOffsets.toString());
+            }
+        }
+
+        KafkaOffsetMapping.saveOffsetStart(segment, startOffsets);
+        KafkaOffsetMapping.saveOffsetEnd(segment, endOffsets);
+
+        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");
+        } catch (IOException e) {
+            logger.error("fail to update cube segment offset", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
new file mode 100644
index 0000000..bb64bf9
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
@@ -0,0 +1,108 @@
+/*
+ * 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.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.time.FastDateFormat;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.kylin.common.util.DateFormat;
+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.mr.HadoopUtil;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.steps.CubingExecutableUtil;
+import org.apache.kylin.job.exception.ExecuteException;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.ExecutableContext;
+import org.apache.kylin.job.execution.ExecuteResult;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class UpdateTimeRangeStep extends AbstractExecutable {
+
+    private static final Logger logger = LoggerFactory.getLogger(UpdateTimeRangeStep.class);
+
+    public UpdateTimeRangeStep() {
+        super();
+    }
+
+    @Override
+    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
+        final CubeManager cubeManager = CubeManager.getInstance(context.getConfig());
+        final CubeInstance cube = cubeManager.getCube(CubingExecutableUtil.getCubeName(this.getParams()));
+        final CubeSegment segment = cube.getSegmentById(CubingExecutableUtil.getSegmentId(this.getParams()));
+        final TblColRef partitionCol = segment.getCubeDesc().getModel().getPartitionDesc().getPartitionDateColumnRef();
+        final String outputPath = this.getParams().get(BatchConstants.CFG_OUTPUT_PATH);
+        final Path outputFile = new Path(outputPath, partitionCol.getName());
+
+        String minValue = null, maxValue = null, currentValue = null;
+        try (FileSystem fs = HadoopUtil.getFileSystem(outputPath); FSDataInputStream inputStream = fs.open(outputFile); BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(inputStream))) {
+            minValue = currentValue = bufferedReader.readLine();
+            while (currentValue != null) {
+                maxValue = currentValue;
+                currentValue = bufferedReader.readLine();
+            }
+        } catch (IOException e) {
+            logger.error("fail to read file " + outputFile, e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+
+        final DataType partitionColType = partitionCol.getType();
+        FastDateFormat dateFormat;
+        if (partitionColType.isDate()) {
+            dateFormat = DateFormat.getDateFormat(DateFormat.DEFAULT_DATE_PATTERN);
+        } else if (partitionColType.isDatetime() || partitionColType.isTimestamp()) {
+            dateFormat = DateFormat.getDateFormat(DateFormat.DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS);
+        } else if (partitionColType.isStringFamily()) {
+            String partitionDateFormat = segment.getCubeDesc().getModel().getPartitionDesc().getPartitionDateFormat();
+            if (StringUtils.isEmpty(partitionDateFormat)) {
+                partitionDateFormat = DateFormat.DEFAULT_DATE_PATTERN;
+            }
+            dateFormat = DateFormat.getDateFormat(partitionDateFormat);
+        } else {
+            return new ExecuteResult(ExecuteResult.State.ERROR, "Type " + partitionColType + " is not valid partition column type");
+        }
+
+        try {
+            long startTime = dateFormat.parse(minValue).getTime();
+            long endTime = dateFormat.parse(maxValue).getTime();
+            CubeUpdate cubeBuilder = new CubeUpdate(cube);
+            segment.setDateRangeStart(startTime);
+            segment.setDateRangeEnd(endTime);
+            cubeBuilder.setToUpdateSegs(segment);
+            cubeManager.updateCube(cubeBuilder);
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+        } catch (Exception e) {
+            logger.error("fail to update cube segment offset", e);
+            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaClusterConfig.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaClusterConfig.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaClusterConfig.java
index 04a66f6..95349c2 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaClusterConfig.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/config/KafkaClusterConfig.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import javax.annotation.Nullable;
 
+import org.apache.kafka.common.protocol.SecurityProtocol;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.RootPersistentEntity;
 import org.apache.kylin.common.persistence.Serializer;
@@ -67,7 +68,7 @@ public class KafkaClusterConfig extends RootPersistentEntity {
             @Nullable
             @Override
             public Broker apply(BrokerConfig input) {
-                return new Broker(input.getId(), input.getHost(), input.getPort());
+                return new Broker(input.getId(), input.getHost(), input.getPort(), SecurityProtocol.PLAINTEXT);
             }
         });
     }


[12/28] kylin git commit: KYLIN-2041 fix IT

Posted by sh...@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/KYLIN-1726-2
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);
     }
 


[27/28] kylin git commit: Revert "Revert "KYLIN-1818 change kafka dependency to provided""

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

This reverts commit dee8f2d073d225a3b8189c66170fd310c1a8d221.


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

Branch: refs/heads/KYLIN-1726-2
Commit: c738f0f265737dd5849f9342fd694e3cef3344b8
Parents: f89e35f
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:56:32 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 2016 +0800

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


http://git-wip-us.apache.org/repos/asf/kylin/blob/c738f0f2/build/bin/find-kafka-dependency.sh
----------------------------------------------------------------------
diff --git a/build/bin/find-kafka-dependency.sh b/build/bin/find-kafka-dependency.sh
index c6b9c24..7349360 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_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" ]
+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" ]
 then
 # works for kafka 8
-    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" ]
+    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" ]
     then
         echo "kafka client lib not found"
         exit 1
     else
-        echo "kafka dependency: $kafka_broker"
+        echo "kafka dependency: $kafka_dependency"
         export kafka_dependency
     fi
 else
-    echo "kafka dependency: $kafka_client"
+    echo "kafka dependency: $kafka_dependency"
     export kafka_dependency
 fi

http://git-wip-us.apache.org/repos/asf/kylin/blob/c738f0f2/build/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/build/bin/kylin.sh b/build/bin/kylin.sh
index 201b5b6..7a9d2a1 100644
--- a/build/bin/kylin.sh
+++ b/build/bin/kylin.sh
@@ -31,6 +31,7 @@ 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" ]
@@ -106,6 +107,7 @@ 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/c738f0f2/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 af2ed9f..a138eec 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 hive dependencies
+        // for kafka dependencies
         if (kylinKafkaDependency != null) {
             kylinKafkaDependency = kylinKafkaDependency.replace(":", ",");
 
-            logger.info("Kafka Dependencies Before Filtered: " + kylinHiveDependency);
+            logger.info("Kafka Dependencies Before Filtered: " + kylinKafkaDependency);
 
             if (kylinDependency.length() > 0)
                 kylinDependency.append(",");


[08/28] kylin git commit: KYLIN-1995: Upgrade deprecated properties in testcases

Posted by sh...@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/KYLIN-1726-2
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>
 


[11/28] kylin git commit: KYLIN-2019: Enable Apache Licence checker for Checkstyle

Posted by sh...@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/KYLIN-1726-2
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;
 


[20/28] kylin git commit: Revert "Revert "rename the streaming_table.json""

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

This reverts commit bec25b4638835301a102141e56e12cd38d09139f.


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

Branch: refs/heads/KYLIN-1726-2
Commit: 8136380f0aa6b0594b2882952e36077c6e474a3c
Parents: 25f8ffc
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:57:11 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 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/8136380f/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/8136380f/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
-}


[22/28] kylin git commit: Revert "Revert "stop scheduler on test finish""

Posted by sh...@apache.org.
Revert "Revert "stop scheduler on test finish""

This reverts commit e604f6527d60b767f8a46a576765cfd205b1efc5.


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

Branch: refs/heads/KYLIN-1726-2
Commit: 8cbffb40477611aa9c06bc5c22787aef83bc28d3
Parents: b1a0f4d
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:58:26 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/job/impl/threadpool/BaseSchedulerTest.java    | 1 +
 .../java/org/apache/kylin/provision/BuildCubeWithEngine.java   | 6 ++++++
 .../java/org/apache/kylin/provision/BuildCubeWithStream.java   | 1 +
 .../java/org/apache/kylin/provision/BuildCubeWithStream2.java  | 1 +
 4 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/8cbffb40/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java b/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
index 4e092a1..97c9f8d 100644
--- a/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
+++ b/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
@@ -55,6 +55,7 @@ public abstract class BaseSchedulerTest extends LocalFileMetadataTestCase {
 
     @After
     public void after() throws Exception {
+        DefaultScheduler.destroyInstance();
         cleanupTestMetadata();
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8cbffb40/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..31cf0eb 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
@@ -79,6 +79,7 @@ public class BuildCubeWithEngine {
             BuildCubeWithEngine buildCubeWithEngine = new BuildCubeWithEngine();
             buildCubeWithEngine.before();
             buildCubeWithEngine.build();
+            buildCubeWithEngine.after();
             logger.info("Build is done");
             afterClass();
             logger.info("Going to exit");
@@ -148,6 +149,11 @@ public class BuildCubeWithEngine {
 
     }
 
+
+    public void after(){
+        DefaultScheduler.destroyInstance();
+    }
+
     public static void afterClass() {
         HBaseMetadataTestCase.staticCleanupTestMetadata();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8cbffb40/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 b7c609e..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
@@ -217,6 +217,7 @@ public class BuildCubeWithStream {
 
     public void after() {
         kafkaServer.stop();
+        DefaultScheduler.destroyInstance();
     }
 
     protected void waitForJob(String jobId) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/8cbffb40/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 d48a473..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
@@ -235,6 +235,7 @@ public class BuildCubeWithStream2 {
 
     public void after() {
         kafkaServer.stop();
+        DefaultScheduler.destroyInstance();
     }
 
     protected void waitForJob(String jobId) {


[19/28] kylin git commit: Revert "Revert "KYLIN-1726 use segment uuid instead of name""

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

This reverts commit 1f4880479cd3132786062723ba70312440de4805.


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

Branch: refs/heads/KYLIN-1726-2
Commit: f89e35f6309c9bec43cc16e68fb35b7490aecc38
Parents: 8431af4
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:56:17 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 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, 36 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/f89e35f6/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 2fadedb..cc2baa5 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
@@ -479,8 +479,11 @@ 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 {
         checkNoBuildingSegment(cube);
 
         CubeSegment newSegment = newSegment(cube, startDate, endDate, startOffset, endOffset);

http://git-wip-us.apache.org/repos/asf/kylin/blob/f89e35f6/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 7f79acc..9e779ab 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,6 +21,8 @@ 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;
 
@@ -145,18 +147,34 @@ 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, true);
+        CubeSegment segment = cubeManager.mergeSegments(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, false);
         DefaultChainedExecutable job = EngineFactory.createBatchMergeJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());
         return job.getId();
     }
 
-    private String refreshSegment(String cubeName, long startOffset, long endOffset) throws Exception {
-        CubeSegment segment = cubeManager.refreshSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset);
+    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());
         DefaultChainedExecutable job = EngineFactory.createBatchCubingJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());
@@ -164,7 +182,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);
+        CubeSegment segment = cubeManager.appendSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, false);
         DefaultChainedExecutable job = EngineFactory.createBatchCubingJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());

http://git-wip-us.apache.org/repos/asf/kylin/blob/f89e35f6/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 cfce137..a5f678f 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_NAME, seg.getName());
+            JobBuilderSupport.appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
             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/f89e35f6/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 decfb60..87d2471 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,7 +33,6 @@ 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;
@@ -70,14 +69,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_NAME);
+            options.addOption(OPTION_SEGMENT_ID);
             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 segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+            String segmentId = getOptionValue(OPTION_SEGMENT_ID);
 
             // ----------------------------------------------------------------------------
             // add metadata to distributed cache
@@ -85,7 +84,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_NAME, segmentName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_ID, segmentId);
             logger.info("Starting: " + job.getJobName());
 
             setJobClasspath(job, cube.getConfig());
@@ -104,11 +103,9 @@ 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.getSegment(segmentName, SegmentStatusEnum.NEW));
+            setupMapper(cube.getSegmentById(segmentId));
             job.setNumReduceTasks(0);
             FileOutputFormat.setOutputPath(job, output);
             FileOutputFormat.setCompressOutput(job, true);

http://git-wip-us.apache.org/repos/asf/kylin/blob/f89e35f6/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 f67fef5..6774c9d 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,6 +105,11 @@ 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);
@@ -119,10 +124,6 @@ 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);


[21/28] kylin git commit: Revert "Revert "KYLIN-1726 update to kafka 0.10""

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

This reverts commit 355e58ba4209ddf945663228688d550bf654c387.


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

Branch: refs/heads/KYLIN-1726-2
Commit: ae3d7e4314bb5d5441d74224967c96a1f05f1b30
Parents: c738f0f
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:56:47 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 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, 53 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/ae3d7e43/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 cc2baa5..5a4b07c 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
@@ -714,12 +714,28 @@ public class CubeManager implements IRealizationProvider {
             return null;
         }
 
-        if (cube.getBuildingSegments().size() > 0) {
-            logger.debug("Cube " + cube.getName() + " has bulding segment, will not trigger merge at this moment");
-            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);
+                    }
+                }
+            }
         }
 
-        List<CubeSegment> ready = cube.getSegments(SegmentStatusEnum.READY);
+        // exclude those already under merging segments
+        readySegs.removeAll(mergingSegs);
 
         long[] timeRanges = cube.getDescriptor().getAutoMergeTimeRanges();
         Arrays.sort(timeRanges);
@@ -727,9 +743,9 @@ public class CubeManager implements IRealizationProvider {
         for (int i = timeRanges.length - 1; i >= 0; i--) {
             long toMergeRange = timeRanges[i];
 
-            for (int s = 0; s < ready.size(); s++) {
-                CubeSegment seg = ready.get(s);
-                Pair<CubeSegment, CubeSegment> p = findMergeOffsetsByDateRange(ready.subList(s, ready.size()), //
+            for (int s = 0; s < readySegs.size(); s++) {
+                CubeSegment seg = readySegs.get(s);
+                Pair<CubeSegment, CubeSegment> p = findMergeOffsetsByDateRange(readySegs.subList(s, readySegs.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/ae3d7e43/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 42b117c..669f53e 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(), req.isForce() || req.isForceMergeEmptySegment());
+        return buildInternal(cubeName, req.getStartTime(), req.getEndTime(), 0, 0, req.getBuildType(), true, 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(), req.isForce());
+        return buildInternal(cubeName, 0, 0, req.getStartSourceOffset(), req.getEndSourceOffset(), req.getBuildType(), false, req.isForce());
     }
 
     private JobInstance buildInternal(String cubeName, long startTime, long endTime, //
-            long startOffset, long endOffset, String buildType, boolean force) {
+            long startOffset, long endOffset, String buildType, boolean strictCheck, 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), force, submitter);
+                    CubeBuildTypeEnum.valueOf(buildType), strictCheck, 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/ae3d7e43/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 5c704ba..8929bf1 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 force, String submitter) throws IOException, JobException {
+            CubeBuildTypeEnum buildType, boolean strictCheck, 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);
+            CubeSegment newSeg = getCubeManager().appendSegment(cube, startDate, endDate, startOffset, endOffset, strictCheck);
             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/ae3d7e43/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 2a86a98..3d26d3d 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,16 +30,15 @@ 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
  */
@@ -49,7 +48,8 @@ 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").create("delay");
+    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 ObjectMapper mapper = new ObjectMapper();
 
@@ -61,6 +61,7 @@ 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() + "'");
@@ -70,7 +71,13 @@ public class KafkaSampleProducer {
         long delay = 0;
         String delayString = optionsHelper.getOptionValue(OPTION_DELAY);
         if (delayString != null) {
-            delay = Long.parseLong(optionsHelper.getOptionValue(OPTION_DELAY));
+            delay = Long.parseLong(delayString);
+        }
+
+        long interval = 1000;
+        String intervalString = optionsHelper.getOptionValue(OPTION_INTERVAL);
+        if (intervalString != null) {
+            interval = Long.parseLong(intervalString);
         }
 
         List<String> countries = new ArrayList();
@@ -95,13 +102,16 @@ public class KafkaSampleProducer {
         devices.add("Other");
 
         Properties props = new Properties();
-        props.put("metadata.broker.list", broker);
-        props.put("serializer.class", "kafka.serializer.StringEncoder");
-        props.put("request.required.acks", "1");
-
-        ProducerConfig config = new ProducerConfig(props);
+        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");
 
-        Producer<String, String> producer = new Producer<String, String>(config);
+        Producer<String, String> producer = new KafkaProducer<>(props);
 
         boolean alive = true;
         Random rnd = new Random();
@@ -114,10 +124,10 @@ public class KafkaSampleProducer {
             record.put("qty", rnd.nextInt(10));
             record.put("currency", "USD");
             record.put("amount", rnd.nextDouble() * 100);
-            KeyedMessage<String, String> data = new KeyedMessage<String, String>(topic, System.currentTimeMillis() + "", mapper.writeValueAsString(record));
+            ProducerRecord<String, String> data = new ProducerRecord<String, String>(topic, System.currentTimeMillis() + "", mapper.writeValueAsString(record));
             System.out.println("Sending 1 message");
             producer.send(data);
-            Thread.sleep(2000);
+            Thread.sleep(interval);
         }
         producer.close();
     }


[18/28] kylin git commit: Revert "Revert "KYLIN-1726 allow job discard itself""

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

This reverts commit 3ae2549ba89e3a2c8ed94a2089678227cf78312d.


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

Branch: refs/heads/KYLIN-1726-2
Commit: f0de02392f4808e08432417cb00f2ab5fa829055
Parents: de2f4e2
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:57:36 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 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, 68 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/f0de0239/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 5a4b07c..3a327f9 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
@@ -357,34 +357,26 @@ 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();
-                        toRemoveResources.add(toRemoveSeg.getStatisticsResourcePath());
-                        found = true;
+                        break;
                     }
                 }
-                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/f0de0239/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 90e4d3c..b4ca469 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,6 +74,8 @@ 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/f0de0239/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..5a57b05 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,6 +119,8 @@ 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/f0de0239/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 760a574..2347e7d 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,6 +49,10 @@ 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/f0de0239/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
new file mode 100644
index 0000000..9362e18
--- /dev/null
+++ b/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java
@@ -0,0 +1,41 @@
+/*
+ * 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/f0de0239/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 df521f9..2baf10a 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,6 +29,7 @@ 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;
@@ -83,6 +84,21 @@ 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();


[13/28] kylin git commit: minor, add dev-support/sync_hbase_cdh_branches.cmd

Posted by sh...@apache.org.
minor, add dev-support/sync_hbase_cdh_branches.cmd


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

Branch: refs/heads/KYLIN-1726-2
Commit: afd3e61eedea2ce175e3fb2844b6118aabe25a6f
Parents: 05afca7
Author: Yang Li <li...@apache.org>
Authored: Tue Sep 27 08:38:42 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Tue Sep 27 08:38:42 2016 +0800

----------------------------------------------------------------------
 dev-support/sync_hbase_cdh_branches.cmd | 19 +++++++++++++++++++
 1 file changed, 19 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/afd3e61e/dev-support/sync_hbase_cdh_branches.cmd
----------------------------------------------------------------------
diff --git a/dev-support/sync_hbase_cdh_branches.cmd b/dev-support/sync_hbase_cdh_branches.cmd
new file mode 100644
index 0000000..0febfbb
--- /dev/null
+++ b/dev-support/sync_hbase_cdh_branches.cmd
@@ -0,0 +1,19 @@
+git checkout master
+git pull
+git reset apache/master --hard
+
+git checkout apache/1.5.x-HBase1.x
+git format-patch -1
+git checkout master
+git am -3 --ignore-whitespace 0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
+git push apache master:1.5.x-HBase1.x -f
+rm 0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
+
+git checkout apache/1.5.x-CDH5.7
+git format-patch -1
+git checkout master
+git am -3 --ignore-whitespace 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
+git push apache master:1.5.x-CDH5.7 -f
+rm 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
+
+git reset apache/master --hard


[09/28] kylin git commit: KYLIN-2041: fix GET Hive Tables parameter definition

Posted by sh...@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/KYLIN-1726-2
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 {


[05/28] kylin git commit: For integration test, don’t use snappy compress for outputformat

Posted by sh...@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/KYLIN-1726-2
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>


[04/28] kylin git commit: KYLIN-1999: Fix UT for the config update

Posted by sh...@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/KYLIN-1726-2
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());
     }
 }


[03/28] kylin git commit: KYLIN-1999, for UT/IT, enable compress

Posted by sh...@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/KYLIN-1726-2
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>


[07/28] kylin git commit: KYLIN-1995: Upgrade deprecated properties for Hadoop 2.6

Posted by sh...@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/KYLIN-1726-2
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>


[23/28] kylin git commit: Revert "Revert "KYLIN-1726 Scalable streaming cubing""

Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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
new file mode 100644
index 0000000..decfb60
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
@@ -0,0 +1,165 @@
+/*
+ * 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.hadoop;
+
+import org.apache.kylin.source.kafka.util.KafkaClient;
+import org.apache.kylin.source.kafka.util.KafkaOffsetMapping;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+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.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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Run a Hadoop Job to process the stream data in kafka;
+ * Modified from the kafka-hadoop-loader in https://github.com/amient/kafka-hadoop-loader
+ */
+public class KafkaFlatTableJob extends AbstractHadoopJob {
+    protected static final Logger logger = LoggerFactory.getLogger(KafkaFlatTableJob.class);
+
+    public static final String CONFIG_KAFKA_PARITION_MIN = "kafka.partition.min";
+    public static final String CONFIG_KAFKA_PARITION_MAX = "kafka.partition.max";
+    public static final String CONFIG_KAFKA_PARITION_START = "kafka.partition.start.";
+    public static final String CONFIG_KAFKA_PARITION_END = "kafka.partition.end.";
+
+    public static final String CONFIG_KAFKA_BROKERS = "kafka.brokers";
+    public static final String CONFIG_KAFKA_TOPIC = "kafka.topic";
+    public static final String CONFIG_KAFKA_TIMEOUT = "kafka.connect.timeout";
+    public static final String CONFIG_KAFKA_BUFFER_SIZE = "kafka.connect.buffer.size";
+    public static final String CONFIG_KAFKA_CONSUMER_GROUP = "kafka.consumer.group";
+    public static final String CONFIG_KAFKA_INPUT_FORMAT = "input.format";
+    public static final String CONFIG_KAFKA_PARSER_NAME = "kafka.parser.name";
+    @Override
+    public int run(String[] args) throws Exception {
+        Options options = new Options();
+
+        try {
+            options.addOption(OPTION_JOB_NAME);
+            options.addOption(OPTION_CUBE_NAME);
+            options.addOption(OPTION_OUTPUT_PATH);
+            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 segmentName = getOptionValue(OPTION_SEGMENT_NAME);
+
+            // ----------------------------------------------------------------------------
+            // add metadata to distributed cache
+            CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+            CubeInstance cube = cubeMgr.getCube(cubeName);
+
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_NAME, cubeName);
+            job.getConfiguration().set(BatchConstants.CFG_CUBE_SEGMENT_NAME, segmentName);
+            logger.info("Starting: " + job.getJobName());
+
+            setJobClasspath(job, cube.getConfig());
+
+            KafkaConfigManager kafkaConfigManager = KafkaConfigManager.getInstance(KylinConfig.getInstanceFromEnv());
+            KafkaConfig kafkaConfig = kafkaConfigManager.getKafkaConfig(cube.getFactTable());
+            String brokers = KafkaClient.getKafkaBrokers(kafkaConfig);
+            String topic = kafkaConfig.getTopic();
+
+            if (brokers == null || brokers.length() == 0 || topic == null) {
+                throw new IllegalArgumentException("Invalid Kafka information, brokers " + brokers + ", topic " + topic);
+            }
+
+            job.getConfiguration().set(CONFIG_KAFKA_BROKERS, brokers);
+            job.getConfiguration().set(CONFIG_KAFKA_TOPIC, topic);
+            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.getSegment(segmentName, SegmentStatusEnum.NEW));
+            job.setNumReduceTasks(0);
+            FileOutputFormat.setOutputPath(job, output);
+            FileOutputFormat.setCompressOutput(job, true);
+            org.apache.log4j.Logger.getRootLogger().info("Output hdfs location: " + output);
+            org.apache.log4j.Logger.getRootLogger().info("Output hdfs compression: " + true);
+            job.getConfiguration().set(BatchConstants.CFG_OUTPUT_PATH, output.toString());
+
+            deletePath(job.getConfiguration(), output);
+
+            attachKylinPropsAndMetadata(cube, job.getConfiguration());
+
+            return waitForCompletion(job);
+
+        } catch (Exception e) {
+            logger.error("error in KafkaFlatTableJob", e);
+            printUsage(options);
+            throw e;
+        } finally {
+            if (job != null)
+                cleanupTempConfFile(job.getConfiguration());
+        }
+
+    }
+
+    private void setupMapper(CubeSegment cubeSeg) throws IOException {
+        // set the segment's offset info to job conf
+        Map<Integer, Long> offsetStart = KafkaOffsetMapping.parseOffsetStart(cubeSeg);
+        Map<Integer, Long> offsetEnd = KafkaOffsetMapping.parseOffsetEnd(cubeSeg);
+
+        Integer minPartition = Collections.min(offsetStart.keySet());
+        Integer maxPartition = Collections.max(offsetStart.keySet());
+        job.getConfiguration().set(CONFIG_KAFKA_PARITION_MIN, minPartition.toString());
+        job.getConfiguration().set(CONFIG_KAFKA_PARITION_MAX, maxPartition.toString());
+
+        for(Integer partition: offsetStart.keySet()) {
+            job.getConfiguration().set(CONFIG_KAFKA_PARITION_START + partition, offsetStart.get(partition).toString());
+            job.getConfiguration().set(CONFIG_KAFKA_PARITION_END + partition, offsetEnd.get(partition).toString());
+        }
+
+        job.setMapperClass(KafkaFlatTableMapper.class);
+        job.setInputFormatClass(KafkaInputFormat.class);
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(Text.class);
+        job.setOutputFormatClass(SequenceFileOutputFormat.class);
+        job.setNumReduceTasks(0);
+    }
+
+    public static void main(String[] args) throws Exception {
+        KafkaFlatTableJob job = new KafkaFlatTableJob();
+        int exitCode = ToolRunner.run(job, args);
+        System.exit(exitCode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableMapper.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableMapper.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableMapper.java
new file mode 100644
index 0000000..995b2d4
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableMapper.java
@@ -0,0 +1,51 @@
+/*
+ * 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.hadoop;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.engine.mr.KylinMapper;
+
+public class KafkaFlatTableMapper extends KylinMapper<LongWritable, BytesWritable, Text, Text> {
+
+    private Text outKey = new Text();
+    private Text outValue = new Text();
+
+    @Override
+    protected void setup(Context context) throws IOException {
+        Configuration conf = context.getConfiguration();
+        bindCurrentConfiguration(conf);
+    }
+
+    @Override
+    public void map(LongWritable key, BytesWritable value, Context context) throws IOException {
+        try {
+            outKey.set(Bytes.toBytes(key.get()));
+            outValue.set(value.getBytes(), 0, value.getLength());
+            context.write(outKey, outValue);
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputFormat.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputFormat.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputFormat.java
new file mode 100644
index 0000000..81f6bac
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputFormat.java
@@ -0,0 +1,98 @@
+/*
+ * 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.hadoop;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import org.apache.kylin.source.kafka.util.KafkaClient;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.PartitionInfo;
+
+/**
+ * Convert Kafka topic to Hadoop InputFormat
+ * Modified from the kafka-hadoop-loader in https://github.com/amient/kafka-hadoop-loader
+ */
+public class KafkaInputFormat extends InputFormat<LongWritable, BytesWritable> {
+
+    @Override
+    public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
+        Configuration conf = context.getConfiguration();
+
+        String brokers = conf.get(KafkaFlatTableJob.CONFIG_KAFKA_BROKERS);
+        String inputTopic = conf.get(KafkaFlatTableJob.CONFIG_KAFKA_TOPIC);
+        String consumerGroup = conf.get(KafkaFlatTableJob.CONFIG_KAFKA_CONSUMER_GROUP);
+        Integer partitionMin = Integer.valueOf(conf.get(KafkaFlatTableJob.CONFIG_KAFKA_PARITION_MIN));
+        Integer partitionMax = Integer.valueOf(conf.get(KafkaFlatTableJob.CONFIG_KAFKA_PARITION_MAX));
+
+        Map<Integer, Long> startOffsetMap = Maps.newHashMap();
+        Map<Integer, Long> endOffsetMap = Maps.newHashMap();
+        for (int i = partitionMin; i <= partitionMax; i++) {
+            String start = conf.get(KafkaFlatTableJob.CONFIG_KAFKA_PARITION_START + i);
+            String end = conf.get(KafkaFlatTableJob.CONFIG_KAFKA_PARITION_END + i);
+            if (start != null && end != null) {
+                startOffsetMap.put(i, Long.valueOf(start));
+                endOffsetMap.put(i, Long.valueOf(end));
+            }
+        }
+
+        List<InputSplit> splits = new ArrayList<InputSplit>();
+        try (KafkaConsumer<String, String> consumer = KafkaClient.getKafkaConsumer(brokers, consumerGroup, null)) {
+            List<PartitionInfo> partitionInfos = consumer.partitionsFor(inputTopic);
+            Preconditions.checkArgument(partitionInfos.size() == startOffsetMap.size(), "partition number mismatch with server side");
+            for (int i = 0; i < partitionInfos.size(); i++) {
+                PartitionInfo partition = partitionInfos.get(i);
+                int partitionId = partition.partition();
+                if (startOffsetMap.containsKey(partitionId) == false) {
+                    throw new IllegalStateException("Partition '" + partitionId + "' not exists.");
+                }
+
+                if (endOffsetMap.get(partitionId) >  startOffsetMap.get(partitionId)) {
+                    InputSplit split = new KafkaInputSplit(
+                            brokers, inputTopic,
+                            partitionId,
+                            startOffsetMap.get(partitionId), endOffsetMap.get(partitionId)
+                    );
+                    splits.add(split);
+                }
+            }
+        }
+        return splits;
+    }
+
+    @Override
+    public RecordReader<LongWritable, BytesWritable> createRecordReader(
+            InputSplit arg0, TaskAttemptContext arg1) throws IOException,
+            InterruptedException {
+        return new KafkaInputRecordReader();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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
new file mode 100644
index 0000000..f67fef5
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputRecordReader.java
@@ -0,0 +1,166 @@
+/*
+ * 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.hadoop;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kylin.common.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Convert Kafka topic to Hadoop InputFormat
+ * Modified from the kafka-hadoop-loader in https://github.com/amient/kafka-hadoop-loader
+ */
+public class KafkaInputRecordReader extends RecordReader<LongWritable, BytesWritable> {
+
+    static Logger log = LoggerFactory.getLogger(KafkaInputRecordReader.class);
+
+    private Configuration conf;
+
+    private KafkaInputSplit split;
+    private Consumer consumer;
+    private String brokers;
+    private String topic;
+
+    private int partition;
+    private long earliestOffset;
+    private long watermark;
+    private long latestOffset;
+
+    private ConsumerRecords<String, String> messages;
+    private Iterator<ConsumerRecord<String, String>> iterator;
+    private LongWritable key;
+    private BytesWritable value;
+
+    private long timeOut = 60000;
+    private long bufferSize = 65536;
+
+    private long numProcessedMessages = 0L;
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
+        initialize(split, context.getConfiguration());
+    }
+
+    public void initialize(InputSplit split, Configuration conf) throws IOException, InterruptedException {
+        this.conf = conf;
+        this.split = (KafkaInputSplit) split;
+        brokers = this.split.getBrokers();
+        topic = this.split.getTopic();
+        partition = this.split.getPartition();
+        watermark = this.split.getOffsetStart();
+
+        if (conf.get(KafkaFlatTableJob.CONFIG_KAFKA_TIMEOUT) != null) {
+            timeOut = Long.parseLong(conf.get(KafkaFlatTableJob.CONFIG_KAFKA_TIMEOUT));
+        }
+        if (conf.get(KafkaFlatTableJob.CONFIG_KAFKA_BUFFER_SIZE) != null) {
+            bufferSize = Long.parseLong(conf.get(KafkaFlatTableJob.CONFIG_KAFKA_BUFFER_SIZE));
+        }
+
+        String consumerGroup = conf.get(KafkaFlatTableJob.CONFIG_KAFKA_CONSUMER_GROUP);
+        consumer = org.apache.kylin.source.kafka.util.KafkaClient.getKafkaConsumer(brokers, consumerGroup, null);
+
+        earliestOffset = this.split.getOffsetStart();
+        latestOffset = this.split.getOffsetEnd();
+        TopicPartition topicPartition = new TopicPartition(topic, partition);
+        consumer.assign(Arrays.asList(topicPartition));
+        log.info("Split {} Topic: {} Broker: {} Partition: {} Start: {} End: {}", new Object[] { this.split, topic, this.split.getBrokers(), partition, earliestOffset, latestOffset });
+    }
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+        if (key == null) {
+            key = new LongWritable();
+        }
+        if (value == null) {
+            value = new BytesWritable();
+        }
+
+        if (messages == null) {
+            log.info("{} fetching offset {} ", topic + ":" + split.getBrokers() + ":" + partition, watermark);
+            TopicPartition topicPartition = new TopicPartition(topic, partition);
+            consumer.seek(topicPartition, watermark);
+            messages = consumer.poll(timeOut);
+            iterator = messages.iterator();
+            if (!iterator.hasNext()) {
+                log.info("No more messages, stop");
+                throw new IOException(String.format("Unexpected ending of stream, expected ending offset %d, but end at %d", latestOffset, watermark));
+            }
+        }
+
+        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);
+            watermark = message.offset() + 1;
+            numProcessedMessages++;
+            if (!iterator.hasNext()) {
+                messages = null;
+                iterator = null;
+            }
+            return true;
+        }
+
+        log.error("Unexpected iterator end.");
+        throw new IOException(String.format("Unexpected ending of stream, expected ending offset %d, but end at %d", latestOffset, watermark));
+    }
+
+    @Override
+    public LongWritable getCurrentKey() throws IOException, InterruptedException {
+        return key;
+    }
+
+    @Override
+    public BytesWritable getCurrentValue() throws IOException, InterruptedException {
+        return value;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        if (watermark >= latestOffset || earliestOffset == latestOffset) {
+            return 1.0f;
+        }
+        return Math.min(1.0f, (watermark - earliestOffset) / (float) (latestOffset - earliestOffset));
+    }
+
+    @Override
+    public void close() throws IOException {
+        log.info("{} num. processed messages {} ", topic + ":" + split.getBrokers() + ":" + partition, numProcessedMessages);
+        consumer.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputSplit.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputSplit.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputSplit.java
new file mode 100644
index 0000000..3261399
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputSplit.java
@@ -0,0 +1,102 @@
+/*
+ * 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.hadoop;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+/**
+ * Convert Kafka topic to Hadoop InputFormat
+ * Modified from the kafka-hadoop-loader in https://github.com/amient/kafka-hadoop-loader
+ */
+public class KafkaInputSplit extends InputSplit implements Writable {
+
+    private String brokers;
+    private String topic;
+    private int partition;
+    private long offsetStart;
+    private long offsetEnd;
+
+    public KafkaInputSplit() {
+    }
+
+    public KafkaInputSplit(String brokers, String topic, int partition, long offsetStart, long offsetEnd) {
+        this.brokers = brokers;
+        this.topic = topic;
+        this.partition = partition;
+        this.offsetStart = offsetStart;
+        this.offsetEnd = offsetEnd;
+    }
+
+    public void readFields(DataInput in) throws IOException {
+        brokers = Text.readString(in);
+        topic = Text.readString(in);
+        partition = in.readInt();
+        offsetStart = in.readLong();
+        offsetEnd = in.readLong();
+    }
+
+    public void write(DataOutput out) throws IOException {
+        Text.writeString(out, brokers);
+        Text.writeString(out, topic);
+        out.writeInt(partition);
+        out.writeLong(offsetStart);
+        out.writeLong(offsetEnd);
+    }
+
+    @Override
+    public long getLength() throws IOException, InterruptedException {
+        return Long.MAX_VALUE;
+    }
+
+    @Override
+    public String[] getLocations() throws IOException, InterruptedException {
+        return new String[]{brokers};
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    public String getTopic() {
+        return topic;
+    }
+
+    public String getBrokers() {
+        return brokers;
+    }
+
+    public long getOffsetStart() {
+        return offsetStart;
+    }
+
+    public long getOffsetEnd() {
+        return offsetEnd;
+    }
+
+    @Override
+    public String toString() {
+        return brokers + "-" + topic + "-" + partition + "-" + offsetStart + "-" + offsetEnd;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java
new file mode 100644
index 0000000..640cc53
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java
@@ -0,0 +1,115 @@
+/*
+ * 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;
+
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kylin.source.kafka.config.BrokerConfig;
+import org.apache.kylin.source.kafka.config.KafkaClusterConfig;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
+
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ */
+public class KafkaClient {
+
+    public static KafkaConsumer getKafkaConsumer(String brokers, String consumerGroup, Properties properties) {
+        Properties props = constructDefaultKafkaConsumerProperties(brokers, consumerGroup, properties);
+        KafkaConsumer<String, String> consumer = new KafkaConsumer<>(props);
+        return consumer;
+    }
+
+    public static KafkaProducer getKafkaProducer(String brokers, Properties properties) {
+        Properties props = constructDefaultKafkaProducerProperties(brokers, properties);
+        KafkaProducer<String, String> producer = new KafkaProducer<String, String>(props);
+        return producer;
+    }
+
+    private static Properties constructDefaultKafkaProducerProperties(String brokers, Properties properties){
+        Properties props = new Properties();
+        props.put("bootstrap.servers", brokers);
+        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+        props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+        props.put("acks", "1");
+        props.put("buffer.memory", 33554432);
+        props.put("retries", 0);
+        props.put("batch.size", 16384);
+        props.put("linger.ms", 50);
+        props.put("timeout.ms", "30000");
+        if (properties != null) {
+            for (Map.Entry entry : properties.entrySet()) {
+                props.put(entry.getKey(), entry.getValue());
+            }
+        }
+        return props;
+    }
+
+    private static Properties constructDefaultKafkaConsumerProperties(String brokers, String consumerGroup, Properties properties) {
+        Properties props = new Properties();
+        props.put("bootstrap.servers", brokers);
+        props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
+        props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
+        props.put("group.id", consumerGroup);
+        props.put("session.timeout.ms", "30000");
+        props.put("enable.auto.commit", "false");
+        if (properties != null) {
+            for (Map.Entry entry : properties.entrySet()) {
+                props.put(entry.getKey(), entry.getValue());
+            }
+        }
+        return props;
+    }
+
+    public static String getKafkaBrokers(KafkaConfig kafkaConfig) {
+        String brokers = null;
+        for (KafkaClusterConfig clusterConfig : kafkaConfig.getKafkaClusterConfigs()) {
+            for (BrokerConfig brokerConfig : clusterConfig.getBrokerConfigs()) {
+                if (brokers == null) {
+                    brokers = brokerConfig.getHost() + ":" + brokerConfig.getPort();
+                } else {
+                    brokers = brokers + "," + brokerConfig.getHost() + ":" + brokerConfig.getPort();
+                }
+            }
+        }
+        return brokers;
+    }
+
+    public static long getEarliestOffset(KafkaConsumer consumer, String topic, int partitionId) {
+
+        TopicPartition topicPartition = new TopicPartition(topic, partitionId);
+        consumer.assign(Arrays.asList(topicPartition));
+        consumer.seekToBeginning(Arrays.asList(topicPartition));
+
+        return consumer.position(topicPartition);
+    }
+
+    public static long getLatestOffset(KafkaConsumer consumer, String topic, int partitionId) {
+
+        TopicPartition topicPartition = new TopicPartition(topic, partitionId);
+        consumer.assign(Arrays.asList(topicPartition));
+        consumer.seekToEnd(Arrays.asList(topicPartition));
+
+        return consumer.position(topicPartition);
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaOffsetMapping.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaOffsetMapping.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaOffsetMapping.java
new file mode 100644
index 0000000..b46e57f
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaOffsetMapping.java
@@ -0,0 +1,97 @@
+/*
+ * 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;
+
+import com.google.common.collect.Maps;
+import org.apache.kylin.cube.CubeSegment;
+
+import java.util.Map;
+
+/**
+ */
+public class KafkaOffsetMapping {
+
+    public static final String OFFSET_START = "kafka.offset.start.";
+    public static final String OFFSET_END = "kafka.offset.end.";
+
+    /**
+     * Get the start offsets for each partition from a segment
+     *
+     * @param segment
+     * @return
+     */
+    public static Map<Integer, Long> parseOffsetStart(CubeSegment segment) {
+        return parseOffset(segment, OFFSET_START);
+    }
+
+    /**
+     * Get the end offsets for each partition from a segment
+     *
+     * @param segment
+     * @return
+     */
+    public static Map<Integer, Long> parseOffsetEnd(CubeSegment segment) {
+        return parseOffset(segment, OFFSET_END);
+    }
+
+    /**
+     * Save the partition start offset to cube segment
+     *
+     * @param segment
+     * @param offsetStart
+     */
+    public static void saveOffsetStart(CubeSegment segment, Map<Integer, Long> offsetStart) {
+        long sourceOffsetStart = 0;
+        for (Integer partition : offsetStart.keySet()) {
+            segment.getAdditionalInfo().put(OFFSET_START + partition, String.valueOf(offsetStart.get(partition)));
+            sourceOffsetStart += offsetStart.get(partition);
+        }
+
+        segment.setSourceOffsetStart(sourceOffsetStart);
+    }
+
+    /**
+     * Save the partition end offset to cube segment
+     *
+     * @param segment
+     * @param offsetEnd
+     */
+    public static void saveOffsetEnd(CubeSegment segment, Map<Integer, Long> offsetEnd) {
+        long sourceOffsetEnd = 0;
+        for (Integer partition : offsetEnd.keySet()) {
+            segment.getAdditionalInfo().put(OFFSET_END + partition, String.valueOf(offsetEnd.get(partition)));
+            sourceOffsetEnd += offsetEnd.get(partition);
+        }
+
+        segment.setSourceOffsetEnd(sourceOffsetEnd);
+    }
+
+    private static Map<Integer, Long> parseOffset(CubeSegment segment, String propertyPrefix) {
+        final Map<Integer, Long> offsetStartMap = Maps.newHashMap();
+        for (String key : segment.getAdditionalInfo().keySet()) {
+            if (key.startsWith(propertyPrefix)) {
+                Integer partition = Integer.valueOf(key.substring(propertyPrefix.length()));
+                Long offset = Long.valueOf(segment.getAdditionalInfo().get(key));
+                offsetStartMap.put(partition, offset);
+            }
+        }
+
+
+        return offsetStartMap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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 919db20..bce9bb9 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
@@ -26,6 +26,8 @@ import java.util.concurrent.ConcurrentMap;
 
 import javax.annotation.Nullable;
 
+import kafka.cluster.BrokerEndPoint;
+import org.apache.kafka.common.protocol.SecurityProtocol;
 import org.apache.kylin.source.kafka.TopicMeta;
 import org.apache.kylin.source.kafka.config.KafkaClusterConfig;
 import org.slf4j.Logger;
@@ -70,13 +72,14 @@ public final class KafkaRequester {
         if (consumerCache.containsKey(key)) {
             return consumerCache.get(key);
         } else {
-            consumerCache.putIfAbsent(key, new SimpleConsumer(broker.host(), broker.port(), timeout, bufferSize, clientId));
+            BrokerEndPoint brokerEndPoint = broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT);
+            consumerCache.putIfAbsent(key, new SimpleConsumer(brokerEndPoint.host(), brokerEndPoint.port(), timeout, bufferSize, clientId));
             return consumerCache.get(key);
         }
     }
 
     private static String createKey(Broker broker, int timeout, int bufferSize, String clientId) {
-        return broker.getConnectionString() + "_" + timeout + "_" + bufferSize + "_" + clientId;
+        return broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).connectionString() + "_" + timeout + "_" + bufferSize + "_" + clientId;
     }
 
     public static TopicMeta getKafkaTopicMeta(KafkaClusterConfig kafkaClusterConfig) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaUtils.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaUtils.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaUtils.java
index 24eaa05..ee5bb20 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaUtils.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaUtils.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 import java.util.Iterator;
 import java.util.Map;
 
+import org.apache.kafka.common.protocol.SecurityProtocol;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.StreamingMessage;
 import org.apache.kylin.source.kafka.StreamingParser;
@@ -55,7 +56,7 @@ public final class KafkaUtils {
             if (partitionMetadata.errorCode() != 0) {
                 logger.warn("PartitionMetadata errorCode: " + partitionMetadata.errorCode());
             }
-            return partitionMetadata.leader();
+            return new Broker(partitionMetadata.leader(), SecurityProtocol.PLAINTEXT);
         } else {
             return null;
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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..f285153 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
@@ -39,7 +39,7 @@ import org.apache.kylin.common.util.CompressionUtils;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.LoggableCachedThreadPool;
 import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.cube.ISegment;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.gridtable.GTInfo;
 import org.apache.kylin.gridtable.GTScanRequest;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
index c318cba..da087c9 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseRPC.java
@@ -31,7 +31,7 @@ import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.ISegment;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.FuzzyKeyEncoder;
 import org.apache.kylin.cube.kv.FuzzyMaskEncoder;

http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/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..5692000 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
@@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.common.util.ShardingHash;
-import org.apache.kylin.cube.ISegment;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.dimension.DimensionEncoding;


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

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

This reverts commit da5ba276b972f8b3c0d220252e74ac2ff73298fc.


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

Branch: refs/heads/KYLIN-1726-2
Commit: 25f8ffc0ee16b1702723d5530b22084a608fed9a
Parents: ae3d7e4
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:57:01 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 2016 +0800

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


http://git-wip-us.apache.org/repos/asf/kylin/blob/25f8ffc0/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 753b389..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
@@ -88,6 +88,7 @@ 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) {
@@ -99,6 +100,9 @@ public class DefaultChainedExecutable extends AbstractExecutable implements Chai
                 if (status == ExecutableState.RUNNING) {
                     hasRunning = true;
                 }
+                if (status == ExecutableState.DISCARDED) {
+                    hasDiscarded = true;
+                }
             }
             if (allSucceed) {
                 setEndTime(System.currentTimeMillis());
@@ -110,6 +114,8 @@ 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/25f8ffc0/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 5dca93f..479f1b8 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,6 +17,10 @@
 */
 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;
@@ -34,6 +38,7 @@ 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;
@@ -101,19 +106,39 @@ public class SeekOffsetStep extends AbstractExecutable {
             }
         }
 
-        KafkaOffsetMapping.saveOffsetStart(segment, startOffsets);
-        KafkaOffsetMapping.saveOffsetEnd(segment, endOffsets);
+        long totalStartOffset = 0, totalEndOffset = 0;
+        for (Long v : startOffsets.values()) {
+            totalStartOffset += v;
+        }
+        for (Long v : endOffsets.values()) {
+            totalEndOffset += v;
+        }
 
-        segment.setName(CubeSegment.makeSegmentName(0, 0, segment.getSourceOffsetStart(), segment.getSourceOffsetEnd()));
-        CubeUpdate cubeBuilder = new CubeUpdate(cube);
-        cubeBuilder.setToUpdateSegs(segment);
-        try {
-            cubeManager.updateCube(cubeBuilder);
+        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());
+            }
             return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
-        } catch (IOException e) {
-            logger.error("fail to update cube segment offset", e);
-            return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
+        } 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");
         }
+
+
     }
 
 }


[15/28] kylin git commit: KYLIN-1962 reorg BuildCubeWithStream test case

Posted by sh...@apache.org.
KYLIN-1962 reorg BuildCubeWithStream test case

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

Branch: refs/heads/KYLIN-1726-2
Commit: 859230d787f9c218f56e56308897b68fb23d8dc4
Parents: ab5563a
Author: shaofengshi <sh...@apache.org>
Authored: Mon Sep 26 18:10:32 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/KylinConfigBase.java    |   2 +-
 .../java/org/apache/kylin/cube/CubeManager.java |  20 +++
 .../java/org/apache/kylin/cube/CubeSegment.java |   7 +-
 .../test_streaming_table_cube_desc.json         |  17 ++-
 .../test_streaming_table_model_desc.json        |   3 +-
 .../kylin/provision/BuildCubeWithStream.java    | 121 +++++++++++------
 .../kylin/provision/BuildCubeWithStream2.java   | 134 -------------------
 7 files changed, 116 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/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 3e41055..838ef97 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
@@ -807,6 +807,6 @@ abstract public class KylinConfigBase implements Serializable {
     }
 
     public int getMaxBuildingSegments() {
-        return Integer.parseInt(getOptional("kylin.cube.building.segment.max", "1"));
+        return Integer.parseInt(getOptional("kylin.cube.building.segment.max", "2"));
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/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 463c8e9..962568c 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
@@ -29,10 +29,12 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 
+import com.google.common.collect.Maps;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.KylinConfigExt;
@@ -476,6 +478,24 @@ public class CubeManager implements IRealizationProvider {
         if (pair.getFirst() == false || pair.getSecond() == false)
             throw new IllegalArgumentException("The new refreshing segment " + newSegment + " does not match any existing segment in cube " + cube);
 
+        if (startOffset > 0 || endOffset > 0) {
+            CubeSegment toRefreshSeg = null;
+            for (CubeSegment cubeSegment : cube.getSegments()) {
+                if (cubeSegment.getSourceOffsetStart() == startOffset && cubeSegment.getSourceOffsetEnd() == endOffset) {
+                    toRefreshSeg = cubeSegment;
+                    break;
+                }
+            }
+
+            if (toRefreshSeg == null) {
+                throw new IllegalArgumentException("For streaming cube, only one segment can be refreshed at one time");
+            }
+
+            Map<String, String> partitionInfo = Maps.newHashMap();
+            partitionInfo.putAll(toRefreshSeg.getAdditionalInfo());
+            newSegment.setAdditionalInfo(partitionInfo);
+        }
+
         CubeUpdate cubeBuilder = new CubeUpdate(cube);
         cubeBuilder.setToAddSegs(newSegment);
         updateCube(cubeBuilder);

http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index afb0d28..d5de47f 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -20,7 +20,6 @@ package org.apache.kylin.cube;
 
 import java.text.SimpleDateFormat;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -106,7 +105,7 @@ public class CubeSegment implements Comparable<CubeSegment>, IBuildable, ISegmen
 
     @JsonProperty("additionalInfo")
     @JsonInclude(JsonInclude.Include.NON_EMPTY)
-    private HashMap<String, String> additionalInfo = new LinkedHashMap<String, String>();
+    private Map<String, String> additionalInfo = new LinkedHashMap<String, String>();
 
     private volatile Map<Long, Short> cuboidBaseShards = Maps.newHashMap();//cuboid id ==> base(starting) shard for this cuboid
 
@@ -543,11 +542,11 @@ public class CubeSegment implements Comparable<CubeSegment>, IBuildable, ISegmen
         this.indexPath = indexPath;
     }
 
-    public HashMap<String, String> getAdditionalInfo() {
+    public Map<String, String> getAdditionalInfo() {
         return additionalInfo;
     }
 
-    public void setAdditionalInfo(HashMap<String, String> additionalInfo) {
+    public void setAdditionalInfo(Map<String, String> additionalInfo) {
         this.additionalInfo = additionalInfo;
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json b/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
index 8279417..640b91c 100644
--- a/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
+++ b/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
@@ -1,30 +1,29 @@
 {
   "uuid" : "901ed15e-7769-4c66-b7ae-fbdc971cd192",
- 
   "name" : "test_streaming_table_cube_desc",
   "description" : "",
   "dimensions" : [ {
-    "name" : "DEFAULT.STREAMING_TABLE.SITE",
+    "name" : "SITE",
     "table" : "DEFAULT.STREAMING_TABLE",
     "column" : "SITE",
     "derived" : null
   }, {
-    "name" : "DEFAULT.STREAMING_TABLE.ITM",
+    "name" : "ITM",
     "table" : "DEFAULT.STREAMING_TABLE",
     "column" : "ITM",
     "derived" : null
   }, {
-    "name" : "TIME",
+    "name" : "DAY_START",
     "table" : "DEFAULT.STREAMING_TABLE",
     "column" : "DAY_START",
     "derived" : null
   }, {
-    "name" : "TIME",
+    "name" : "HOUR_START",
     "table" : "DEFAULT.STREAMING_TABLE",
     "column" : "HOUR_START",
     "derived" : null
   }, {
-    "name" : "TIME",
+    "name" : "MINUTE_START",
     "table" : "DEFAULT.STREAMING_TABLE",
     "column" : "MINUTE_START",
     "derived" : null
@@ -68,13 +67,13 @@
   } ],
   "rowkey" : {
     "rowkey_columns" : [ {
-      "column" : "DAY_START",
+      "column" : "MINUTE_START",
       "encoding" : "dict"
     }, {
       "column" : "HOUR_START",
       "encoding" : "dict"
     }, {
-      "column" : "MINUTE_START",
+      "column" : "DAY_START",
       "encoding" : "dict"
     }, {
       "column" : "SITE",
@@ -107,7 +106,7 @@
   } ],
   "override_kylin_properties": {
     "kylin.cube.algorithm": "inmem",
-    "kylin.cube.building.segment.max": "3"
+    "kylin.cube.building.segment.max": "5"
   },
   "notify_list" : [ ],
   "status_need_notify" : [ ],

http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json b/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json
index e6977e1..23b10f7 100644
--- a/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json
+++ b/examples/test_case_data/localmeta/model_desc/test_streaming_table_model_desc.json
@@ -1,5 +1,4 @@
 {
- 
   "uuid": "ff527b94-f860-44c3-8452-93b177888732",
   "name": "test_streaming_table_model_desc",
   "dimensions": [
@@ -23,7 +22,7 @@
   "fact_table": "DEFAULT.STREAMING_TABLE",
   "filter_condition": null,
   "partition_desc": {
-    "partition_date_column": "DEFAULT.STREAMING_TABLE.minute_start",
+    "partition_date_column": "DEFAULT.STREAMING_TABLE.MINUTE_START",
     "partition_date_start": 0,
     "partition_type": "APPEND"
   }

http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/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 dfcedfb..23d7ca8 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
@@ -20,12 +20,21 @@ package org.apache.kylin.provision;
 
 import java.io.File;
 import java.io.IOException;
+import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+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;
+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;
@@ -47,6 +56,7 @@ 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;
@@ -55,6 +65,8 @@ import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static java.lang.Thread.sleep;
+
 /**
  *  for streaming cubing case "test_streaming_table"
  */
@@ -70,6 +82,7 @@ public class BuildCubeWithStream {
     private KafkaConfig kafkaConfig;
     private MockKafka kafkaServer;
     protected static boolean fastBuildMode = false;
+    private boolean generateData = true;
 
     public void before() throws Exception {
         deployEnv();
@@ -139,44 +152,91 @@ public class BuildCubeWithStream {
 
     public void build() throws Exception {
         clearSegment(cubeName);
-        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
-        f.setTimeZone(TimeZone.getTimeZone("GMT"));
-        long date1 = 0;
-        long date2 = f.parse("2013-01-01").getTime();
+        new Thread(new Runnable() {
+            @Override
+            public void run() {
+                SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+                f.setTimeZone(TimeZone.getTimeZone("GMT"));
+                long dateStart = 0;
+                try {
+                    dateStart = f.parse("2012-01-01").getTime();
+                } catch (ParseException e) {
+                }
+                Random rand = new Random();
+                while (generateData == true) {
+                    long dateEnd = dateStart + 7 * 24 * 3600000;
+                    try {
+                        generateStreamData(dateStart, dateEnd, rand.nextInt(100));
+                        dateStart = dateEnd;
+                        sleep(rand.nextInt(rand.nextInt(100 * 1000))); // wait random time
+                    } catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+        }).start();
+        ExecutorService executorService = Executors.newCachedThreadPool();
+
+        List<FutureTask<ExecutableState>> futures = Lists.newArrayList();
+        for (int i = 0; i < 5; i++) {
+            Thread.sleep(2 * 60 * 1000); // wait for new messages
+            FutureTask futureTask = new FutureTask(new Callable<ExecutableState>() {
+                @Override
+                public ExecutableState call() {
+                    ExecutableState result = null;
+                    try {
+                        result = buildSegment(cubeName, 0, Long.MAX_VALUE);
+                    } catch (Exception e) {
+                        // previous build hasn't been started, or other case.
+                        e.printStackTrace();
+                    }
+
+                    return result;
+                }
+            });
 
-        int numberOfRecrods1 = 10000;
-        generateStreamData(date1, date2, numberOfRecrods1);
-        ExecutableState result = buildSegment(cubeName, 0, Long.MAX_VALUE);
-        Assert.assertTrue(result == ExecutableState.SUCCEED);
+            executorService.submit(futureTask);
+            futures.add(futureTask);
+        }
 
-        if (fastBuildMode == false) {
-            long date3 = f.parse("2013-04-01").getTime();
-            int numberOfRecords2 = 5000;
-            generateStreamData(date2, date3, numberOfRecords2);
-            result = buildSegment(cubeName, 0, Long.MAX_VALUE);
-            Assert.assertTrue(result == ExecutableState.SUCCEED);
+        generateData = false; // stop generating message to kafka
+        executorService.shutdown();
+        int succeedBuild = 0;
+        for (int i = 0; i < futures.size(); i++) {
+            ExecutableState result = futures.get(i).get(20, TimeUnit.MINUTES);
+            logger.info("Checking building task " + i + " whose state is " + result);
+            Assert.assertTrue(result == null || result == ExecutableState.SUCCEED || result == ExecutableState.DISCARDED );
+            if (result == ExecutableState.SUCCEED)
+                succeedBuild++;
+        }
+
+        logger.info(succeedBuild + " build jobs have been successfully completed.");
+        List<CubeSegment> segments = cubeManager.getCube(cubeName).getSegments(SegmentStatusEnum.READY);
+        Assert.assertTrue(segments.size() == succeedBuild);
 
+
+        if (fastBuildMode == false) {
             //empty build
-            result = buildSegment(cubeName, 0, Long.MAX_VALUE);
+            ExecutableState result = buildSegment(cubeName, 0, Long.MAX_VALUE);
             Assert.assertTrue(result == ExecutableState.DISCARDED);
 
+            long endOffset = segments.get(segments.size() - 1).getSourceOffsetEnd();
             //merge
-            result = mergeSegment(cubeName, 0, 15000);
+            result = mergeSegment(cubeName, 0, endOffset);
             Assert.assertTrue(result == ExecutableState.SUCCEED);
 
-            List<CubeSegment> segments = cubeManager.getCube(cubeName).getSegments();
+            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);
+            refreshSegment(cubeName, toRefreshSeg.getSourceOffsetStart(), toRefreshSeg.getSourceOffsetEnd());
             segments = cubeManager.getCube(cubeName).getSegments();
             Assert.assertTrue(segments.size() == 1);
         }
-
     }
 
+
     private ExecutableState mergeSegment(String cubeName, long startOffset, long endOffset) throws Exception {
         CubeSegment segment = cubeManager.mergeSegments(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, false);
         DefaultChainedExecutable job = EngineFactory.createBatchMergeJob(segment, "TEST");
@@ -185,14 +245,8 @@ public class BuildCubeWithStream {
         return job.getStatus();
     }
 
-    private String refreshSegment(String cubeName, long startOffset, long endOffset, HashMap<String, String> partitionOffsetMap) throws Exception {
+    private String refreshSegment(String cubeName, long startOffset, long endOffset) throws Exception {
         CubeSegment segment = cubeManager.refreshSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset);
-        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());
         DefaultChainedExecutable job = EngineFactory.createBatchCubingJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());
@@ -209,8 +263,8 @@ public class BuildCubeWithStream {
 
     protected void deployEnv() throws IOException {
         DeployUtil.overrideJobJarLocations();
-//        DeployUtil.initCliWorkDir();
-//        DeployUtil.deployMetadata();
+        //        DeployUtil.initCliWorkDir();
+        //        DeployUtil.deployMetadata();
     }
 
     public static void beforeClass() throws Exception {
@@ -265,13 +319,4 @@ public class BuildCubeWithStream {
         }
 
     }
-
-    protected int cleanupOldStorage() throws Exception {
-        String[] args = { "--delete", "true" };
-
-        //        KapStorageCleanupCLI cli = new KapStorageCleanupCLI();
-        //        cli.execute(args);
-        return 0;
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/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
deleted file mode 100644
index d8c857f..0000000
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
+++ /dev/null
@@ -1,134 +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.provision;
-
-import java.io.IOException;
-import java.text.SimpleDateFormat;
-import java.util.List;
-import java.util.Random;
-import java.util.TimeZone;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.Lists;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.job.execution.ExecutableState;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.junit.Assert;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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 {
-
-    private static final Logger logger = LoggerFactory.getLogger(BuildCubeWithStream2.class);
-    private boolean generateData = true;
-
-    @Override
-    public void build() throws Exception {
-        clearSegment(cubeName);
-        SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
-        f.setTimeZone(TimeZone.getTimeZone("GMT"));
-        final long date1 = 0;
-        final long date2 = f.parse("2013-01-01").getTime();
-
-        new Thread(new Runnable() {
-            @Override
-            public void run() {
-
-                Random rand = new Random();
-                while (generateData == true) {
-                    try {
-                        generateStreamData(date1, date2, rand.nextInt(100));
-                        sleep(rand.nextInt(rand.nextInt(100 * 1000))); // wait random time, from 0 to 100 seconds
-                    } catch (IOException e) {
-                        e.printStackTrace();
-                    } catch (InterruptedException e) {
-                        e.printStackTrace();
-                    }
-                }
-            }
-        }).start();
-        ExecutorService executorService = Executors.newFixedThreadPool(4);
-
-        List<FutureTask<ExecutableState>> futures = Lists.newArrayList();
-        for (int i = 0; i < 5; i++) {
-            Thread.sleep(2 * 60 * 1000); // sleep 2 mintues
-            FutureTask futureTask = new FutureTask(new Callable<ExecutableState>() {
-                @Override
-                public ExecutableState call() {
-                    ExecutableState result = null;
-                    try {
-                        result = buildSegment(cubeName, 0, Long.MAX_VALUE);
-                    } catch (Exception e) {
-                        e.printStackTrace();
-                    }
-
-                    return result;
-                }
-            });
-
-            executorService.submit(futureTask);
-            futures.add(futureTask);
-        }
-
-        generateData = false; // stop generating message to kafka
-        executorService.shutdown();
-        int succeedBuild = 0;
-        for (int i = 0; i < futures.size(); i++) {
-            ExecutableState result = futures.get(i).get(20, TimeUnit.MINUTES);
-            logger.info("Checking building task " + i + " whose state is " + result);
-            Assert.assertTrue(result == null || result == ExecutableState.SUCCEED || result == ExecutableState.DISCARDED );
-            if (result == ExecutableState.SUCCEED)
-                succeedBuild++;
-        }
-
-        logger.info(succeedBuild + " build jobs have been successfully completed.");
-        List<CubeSegment> segments = cubeManager.getCube(cubeName).getSegments(SegmentStatusEnum.READY);
-        Assert.assertTrue(segments.size() == succeedBuild);
-
-    }
-
-    public static void main(String[] args) throws Exception {
-        try {
-            beforeClass();
-
-            BuildCubeWithStream2 buildCubeWithStream = new BuildCubeWithStream2();
-            buildCubeWithStream.before();
-            buildCubeWithStream.build();
-            logger.info("Build is done");
-            buildCubeWithStream.after();
-            afterClass();
-            logger.info("Going to exit");
-            System.exit(0);
-        } catch (Exception e) {
-            logger.error("error", e);
-            System.exit(1);
-        }
-
-    }
-
-}


[17/28] kylin git commit: Revert "Revert "refactor BuildCubeWithStream""

Posted by sh...@apache.org.
Revert "Revert "refactor BuildCubeWithStream""

This reverts commit 8e9c4550bb562b497442b17eec6485ae96e848d8.


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

Branch: refs/heads/KYLIN-1726-2
Commit: be18158dcc5ce739c272b9345d3b2296c3936ee3
Parents: 8cbffb4
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:58:43 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 2016 +0800

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


http://git-wip-us.apache.org/repos/asf/kylin/blob/be18158d/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 9b282e3..9e9df05 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -187,6 +187,7 @@ 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;
@@ -194,13 +195,15 @@ public class DeployUtil {
             }
             IOUtils.copy(in, out);
             IOUtils.closeQuietly(in);
+            IOUtils.closeQuietly(out);
 
             store.deleteResource(factTablePath);
-            in = new FileInputStream(tmpFile);
-            store.putResource(factTablePath, in, System.currentTimeMillis());
+            tempIn = new FileInputStream(tmpFile);
+            store.putResource(factTablePath, tempIn, System.currentTimeMillis());
         } finally {
             IOUtils.closeQuietly(out);
             IOUtils.closeQuietly(in);
+            IOUtils.closeQuietly(tempIn);
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/be18158d/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 6e5313f..bfe1d0a 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);
 
-    private CubeManager cubeManager;
+    protected CubeManager cubeManager;
     private DefaultScheduler scheduler;
     protected ExecutableManager jobService;
-    private static final String cubeName = "test_streaming_table_cube";
+    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());
     }
 
-    private void generateStreamData(long startTime, long endTime, int numberOfRecords) throws IOException {
+    protected 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");
     }
 
-    private void clearSegment(String cubeName) throws Exception {
+    protected 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();
     }
 
-    private ExecutableState buildSegment(String cubeName, long startOffset, long endOffset) throws Exception {
+    protected 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/be18158d/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 2812446..7959701 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,13 +18,11 @@
 
 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;
@@ -32,32 +30,9 @@ 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;
@@ -67,79 +42,12 @@ import static java.lang.Thread.sleep;
 /**
  *  for streaming cubing case "test_streaming_table", using multiple threads to build it concurrently.
  */
-public class BuildCubeWithStream2 {
+public class BuildCubeWithStream2 extends BuildCubeWithStream {
 
     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;
 
-    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);
-    }
-
+    @Override
     public void build() throws Exception {
         clearSegment(cubeName);
         SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
@@ -204,55 +112,6 @@ public class BuildCubeWithStream2 {
 
     }
 
-
-    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();


[10/28] kylin git commit: KYLIN-2019: Enable Apache Licence checker for Checkstyle

Posted by sh...@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;


[14/28] kylin git commit: rename license header file

Posted by sh...@apache.org.
rename license header file


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

Branch: refs/heads/KYLIN-1726-2
Commit: a00d1e37483a5f15bf8f469fb1ed23bba1aa0011
Parents: afd3e61
Author: shaofengshi <sh...@apache.org>
Authored: Tue Sep 27 09:55:03 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 09:55:03 2016 +0800

----------------------------------------------------------------------
 dev-support/checkstyle-apache-header.txt | 17 +++++++++++++++++
 dev-support/checkstyle-java-header.txt   | 17 -----------------
 dev-support/checkstyle.xml               |  2 +-
 3 files changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a00d1e37/dev-support/checkstyle-apache-header.txt
----------------------------------------------------------------------
diff --git a/dev-support/checkstyle-apache-header.txt b/dev-support/checkstyle-apache-header.txt
new file mode 100644
index 0000000..3740126
--- /dev/null
+++ b/dev-support/checkstyle-apache-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/a00d1e37/dev-support/checkstyle-java-header.txt
----------------------------------------------------------------------
diff --git a/dev-support/checkstyle-java-header.txt b/dev-support/checkstyle-java-header.txt
deleted file mode 100644
index 3740126..0000000
--- a/dev-support/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/a00d1e37/dev-support/checkstyle.xml
----------------------------------------------------------------------
diff --git a/dev-support/checkstyle.xml b/dev-support/checkstyle.xml
index ec36760..45325bf 100644
--- a/dev-support/checkstyle.xml
+++ b/dev-support/checkstyle.xml
@@ -21,7 +21,7 @@
         "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="headerFile" value="dev-support/checkstyle-apache-header.txt"/>
         <property name="fileExtensions" value="java"/>
         <property name="ignoreLines" value="9, 11, 17"/>
     </module>


[25/28] kylin git commit: KYLIN-1762 fix query test error

Posted by sh...@apache.org.
KYLIN-1762 fix query test error

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

Branch: refs/heads/KYLIN-1726-2
Commit: ab5563a8ec060fba48ec8f43244bed6f887b0e83
Parents: be18158
Author: shaofengshi <sh...@apache.org>
Authored: Sun Sep 25 21:41:37 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 2016 +0800

----------------------------------------------------------------------
 .../kylin/job/streaming/Kafka10DataLoader.java  |  2 +-
 .../cube/model/CubeJoinedFlatTableDesc.java     |  2 +-
 .../mr/steps/FactDistinctColumnPartitioner.java |  3 ++
 .../engine/mr/steps/FactDistinctColumnsJob.java |  2 +-
 .../mr/steps/FactDistinctColumnsReducer.java    | 38 +++++++++-----
 .../mr/steps/FactDistinctHiveColumnsMapper.java | 49 ++++++++++++++++--
 .../kafka/DEFAULT.STREAMING_TABLE.json          |  1 +
 .../kylin/provision/BuildCubeWithStream.java    | 52 ++++++++++++--------
 .../kylin/provision/BuildCubeWithStream2.java   |  4 +-
 .../apache/kylin/query/ITKylinQueryTest.java    |  3 ++
 .../org/apache/kylin/query/KylinTestBase.java   |  2 +-
 .../apache/kylin/source/kafka/KafkaMRInput.java |  2 +-
 .../source/kafka/TimedJsonStreamParser.java     |  7 +--
 13 files changed, 117 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java b/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
index 2b299cc..8c548be 100644
--- a/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
+++ b/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
@@ -65,7 +65,7 @@ public class Kafka10DataLoader extends StreamDataLoader {
         props.put("retry.backoff.ms", "1000");
         KafkaProducer producer = KafkaClient.getKafkaProducer(brokerList, props);
 
-        for (int i = 0; i < messages.size(); ++i) {
+        for (int i = 0; i < messages.size(); i++) {
             ProducerRecord<String, String> keyedMessage = new ProducerRecord<String, String>(clusterConfig.getTopic(), String.valueOf(i), messages.get(i));
             producer.send(keyedMessage);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
index 6ca89c8..5cd4f1d 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
@@ -143,7 +143,7 @@ public class CubeJoinedFlatTableDesc implements IJoinedFlatTableDesc {
     public int getColumnIndex(TblColRef colRef) {
         Integer index = columnIndexMap.get(colRef);
         if (index == null)
-            throw new IllegalArgumentException("Column " + colRef.toString() + " wasn't found on flat table.");
+            return -1;
 
         return index.intValue();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnPartitioner.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnPartitioner.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnPartitioner.java
index a631cf4..6973c4b 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnPartitioner.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnPartitioner.java
@@ -34,6 +34,9 @@ public class FactDistinctColumnPartitioner extends Partitioner<Text, Text> {
         if (key.getBytes()[0] == FactDistinctHiveColumnsMapper.MARK_FOR_HLL) {
             // the last reducer is for merging hll
             return numReduceTasks - 1;
+        } else if (key.getBytes()[0] == FactDistinctHiveColumnsMapper.MARK_FOR_PARTITION_COL) {
+            // the last reducer is for merging hll
+            return numReduceTasks - 2;
         } else {
             int colIndex = BytesUtil.readUnsigned(key.getBytes(), 0, 1);
             return colIndex;

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
index a6c4d30..a9cc17f 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
@@ -101,7 +101,7 @@ public class FactDistinctColumnsJob extends AbstractHadoopJob {
                 System.out.println("Found segment " + segment);
             }
             setupMapper(cube.getSegmentById(segmentID));
-            setupReducer(output, "true".equalsIgnoreCase(statistics_enabled) ? columnsNeedDict.size() + 1 : columnsNeedDict.size());
+            setupReducer(output, "true".equalsIgnoreCase(statistics_enabled) ? columnsNeedDict.size() + 2 : columnsNeedDict.size());
 
             attachKylinPropsAndMetadata(cube, job.getConfiguration());
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
index 0c13df7..2889ba8 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
@@ -65,7 +65,7 @@ public class FactDistinctColumnsReducer extends KylinReducer<Text, Text, NullWri
     private List<ByteArray> colValues;
     private TblColRef col = null;
     private boolean isStatistics = false;
-    private boolean outputTouched = false;
+    private boolean isPartitionCol = false;
     private KylinConfig cubeConfig;
     protected static final Logger logger = LoggerFactory.getLogger(FactDistinctColumnsReducer.class);
 
@@ -92,25 +92,25 @@ public class FactDistinctColumnsReducer extends KylinReducer<Text, Text, NullWri
             baseCuboidRowCountInMappers = Lists.newArrayList();
             cuboidHLLMap = Maps.newHashMap();
             samplingPercentage = Integer.parseInt(context.getConfiguration().get(BatchConstants.CFG_STATISTICS_SAMPLING_PERCENT));
+        } else if (collectStatistics && (taskId == numberOfTasks - 2)) {
+            // partition col
+            isStatistics = false;
+            isPartitionCol = true;
+            col = cubeDesc.getModel().getPartitionDesc().getPartitionDateColumnRef();
+            colValues = Lists.newLinkedList();
         } else {
             // col
             isStatistics = false;
+            isPartitionCol = false;
             col = columnList.get(taskId);
-            colValues = Lists.newArrayList();
+            colValues = Lists.newLinkedList();
         }
     }
 
     @Override
     public void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
 
-        if (isStatistics == false) {
-            colValues.add(new ByteArray(Bytes.copy(key.getBytes(), 1, key.getLength() - 1)));
-            if (colValues.size() == 1000000) { //spill every 1 million
-                logger.info("spill values to disk...");
-                outputDistinctValues(col, colValues, context);
-                colValues.clear();
-            }
-        } else {
+        if (isStatistics == true) {
             // for hll
             long cuboidId = Bytes.toLong(key.getBytes(), 1, Bytes.SIZEOF_LONG);
             for (Text value : values) {
@@ -130,6 +130,21 @@ public class FactDistinctColumnsReducer extends KylinReducer<Text, Text, NullWri
                     cuboidHLLMap.put(cuboidId, hll);
                 }
             }
+        } else if (isPartitionCol == true) {
+            // for partition col min/max value
+            ByteArray value = new ByteArray(Bytes.copy(key.getBytes(), 1, key.getLength() - 1));
+            if (colValues.size() > 1) {
+                colValues.set(1, value);
+            } else {
+                colValues.add(value);
+            }
+        } else {
+            colValues.add(new ByteArray(Bytes.copy(key.getBytes(), 1, key.getLength() - 1)));
+            if (colValues.size() == 1000000) { //spill every 1 million
+                logger.info("spill values to disk...");
+                outputDistinctValues(col, colValues, context);
+                colValues.clear();
+            }
         }
 
     }
@@ -156,7 +171,6 @@ public class FactDistinctColumnsReducer extends KylinReducer<Text, Text, NullWri
             }
         } finally {
             IOUtils.closeQuietly(out);
-            outputTouched = true;
         }
     }
 
@@ -164,7 +178,7 @@ public class FactDistinctColumnsReducer extends KylinReducer<Text, Text, NullWri
     protected void cleanup(Context context) throws IOException, InterruptedException {
 
         if (isStatistics == false) {
-            if (!outputTouched || colValues.size() > 0) {
+            if (colValues.size() > 0) {
                 outputDistinctValues(col, colValues, context);
                 colValues.clear();
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/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 5e278f8..86ef487 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
@@ -35,6 +35,7 @@ import com.google.common.collect.Lists;
 import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hasher;
 import com.google.common.hash.Hashing;
+import org.apache.kylin.metadata.model.TblColRef;
 
 /**
  */
@@ -52,8 +53,12 @@ public class FactDistinctHiveColumnsMapper<KEYIN> extends FactDistinctColumnsMap
     private ByteArray[] row_hashcodes = null;
     private ByteBuffer keyBuffer;
     private static final Text EMPTY_TEXT = new Text();
+    public static final byte MARK_FOR_PARTITION_COL = (byte) 0xFE;
     public static final byte MARK_FOR_HLL = (byte) 0xFF;
 
+    private int partitionColumnIndex = -1;
+    private boolean needFetchPartitionCol = true;
+
     @Override
     protected void setup(Context context) throws IOException {
         super.setup(context);
@@ -81,6 +86,26 @@ public class FactDistinctHiveColumnsMapper<KEYIN> extends FactDistinctColumnsMap
             for (int i = 0; i < nRowKey; i++) {
                 row_hashcodes[i] = new ByteArray();
             }
+
+            TblColRef partitionColRef = cubeDesc.getModel().getPartitionDesc().getPartitionDateColumnRef();
+            if (partitionColRef != null) {
+                partitionColumnIndex = intermediateTableDesc.getColumnIndex(partitionColRef);
+            }
+
+            // check whether need fetch the partition col values
+            if (partitionColumnIndex < 0) {
+                // if partition col not on cube, no need
+                needFetchPartitionCol = false;
+            } else {
+                for (int x : dictionaryColumnIndex) {
+                    if (x == partitionColumnIndex) {
+                        // if partition col already build dict, no need
+                        needFetchPartitionCol = false;
+                        break;
+                    }
+                }
+            }
+
         }
     }
 
@@ -108,24 +133,38 @@ public class FactDistinctHiveColumnsMapper<KEYIN> extends FactDistinctColumnsMap
     @Override
     public void map(KEYIN key, Object record, Context context) throws IOException, InterruptedException {
         String[] row = flatTableInputFormat.parseMapperInput(record);
+
+        keyBuffer.clear();
         try {
             for (int i = 0; i < factDictCols.size(); i++) {
                 String fieldValue = row[dictionaryColumnIndex[i]];
                 if (fieldValue == null)
                     continue;
-
-                keyBuffer.clear();
+                int offset = keyBuffer.position();
                 keyBuffer.put(Bytes.toBytes(i)[3]); // one byte is enough
                 keyBuffer.put(Bytes.toBytes(fieldValue));
-                outputKey.set(keyBuffer.array(), 0, keyBuffer.position());
+                outputKey.set(keyBuffer.array(), offset, keyBuffer.position() - offset);
                 context.write(outputKey, EMPTY_TEXT);
             }
         } catch (Exception ex) {
             handleErrorRecord(row, ex);
         }
 
-        if (collectStatistics && rowCount < samplingPercentage) {
-            putRowKeyToHLL(row);
+        if (collectStatistics) {
+            if (rowCount < samplingPercentage) {
+                putRowKeyToHLL(row);
+            }
+
+            if (needFetchPartitionCol == true) {
+                String fieldValue = row[partitionColumnIndex];
+                if (fieldValue != null) {
+                    int offset = keyBuffer.position();
+                    keyBuffer.put(MARK_FOR_PARTITION_COL);
+                    keyBuffer.put(Bytes.toBytes(fieldValue));
+                    outputKey.set(keyBuffer.array(), offset, keyBuffer.position() - offset);
+                    context.write(outputKey, EMPTY_TEXT);
+                }
+            }
         }
 
         if (rowCount++ == 100)

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/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
index 6a64cce..e3ac2d6 100644
--- a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
+++ b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
@@ -6,6 +6,7 @@
   "timeout": 60000,
   "bufferSize": 65536,
   "parserName": "org.apache.kylin.source.kafka.TimedJsonStreamParser",
+  "parserProperties": "tsColName=timestamp",
   "last_modified": 0,
   "clusters": [
     {

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/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..dfcedfb 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
@@ -69,10 +69,19 @@ public class BuildCubeWithStream {
 
     private KafkaConfig kafkaConfig;
     private MockKafka kafkaServer;
+    protected static boolean fastBuildMode = false;
 
     public void before() throws Exception {
         deployEnv();
 
+        String fastModeStr = System.getProperty("fastBuildMode");
+        if (fastModeStr != null && fastModeStr.equalsIgnoreCase("true")) {
+            fastBuildMode = true;
+            logger.info("Will use fast build mode");
+        } else {
+            logger.info("Will not use fast build mode");
+        }
+
         final KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
         jobService = ExecutableManager.getInstance(kylinConfig);
         scheduler = DefaultScheduler.createInstance();
@@ -139,29 +148,32 @@ public class BuildCubeWithStream {
         generateStreamData(date1, date2, numberOfRecrods1);
         ExecutableState result = buildSegment(cubeName, 0, Long.MAX_VALUE);
         Assert.assertTrue(result == ExecutableState.SUCCEED);
-        long date3 = f.parse("2013-04-01").getTime();
-        int numberOfRecords2 = 5000;
-        generateStreamData(date2, date3, numberOfRecords2);
-        result = buildSegment(cubeName, 0, Long.MAX_VALUE);
-        Assert.assertTrue(result == ExecutableState.SUCCEED);
 
-        //empty build
-        result = buildSegment(cubeName, 0, Long.MAX_VALUE);
-        Assert.assertTrue(result == ExecutableState.DISCARDED);
+        if (fastBuildMode == false) {
+            long date3 = f.parse("2013-04-01").getTime();
+            int numberOfRecords2 = 5000;
+            generateStreamData(date2, date3, numberOfRecords2);
+            result = buildSegment(cubeName, 0, Long.MAX_VALUE);
+            Assert.assertTrue(result == ExecutableState.SUCCEED);
 
-        //merge
-        result = mergeSegment(cubeName, 0, 15000);
-        Assert.assertTrue(result == ExecutableState.SUCCEED);
+            //empty build
+            result = buildSegment(cubeName, 0, Long.MAX_VALUE);
+            Assert.assertTrue(result == ExecutableState.DISCARDED);
+
+            //merge
+            result = mergeSegment(cubeName, 0, 15000);
+            Assert.assertTrue(result == ExecutableState.SUCCEED);
 
-        List<CubeSegment> segments = cubeManager.getCube(cubeName).getSegments();
-        Assert.assertTrue(segments.size() == 1);
+            List<CubeSegment> segments = cubeManager.getCube(cubeName).getSegments();
+            Assert.assertTrue(segments.size() == 1);
 
-        CubeSegment toRefreshSeg = segments.get(0);
-        HashMap<String, String> partitionOffsetMap = toRefreshSeg.getAdditionalInfo();
+            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);
+            refreshSegment(cubeName, toRefreshSeg.getSourceOffsetStart(), toRefreshSeg.getSourceOffsetEnd(), partitionOffsetMap);
+            segments = cubeManager.getCube(cubeName).getSegments();
+            Assert.assertTrue(segments.size() == 1);
+        }
 
     }
 
@@ -197,8 +209,8 @@ public class BuildCubeWithStream {
 
     protected void deployEnv() throws IOException {
         DeployUtil.overrideJobJarLocations();
-        //DeployUtil.initCliWorkDir();
-        //DeployUtil.deployMetadata();
+//        DeployUtil.initCliWorkDir();
+//        DeployUtil.deployMetadata();
     }
 
     public static void beforeClass() throws Exception {

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/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..d8c857f 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
@@ -45,7 +45,7 @@ import static java.lang.Thread.sleep;
 public class BuildCubeWithStream2 extends BuildCubeWithStream {
 
     private static final Logger logger = LoggerFactory.getLogger(BuildCubeWithStream2.class);
-    private static boolean generateData = true;
+    private boolean generateData = true;
 
     @Override
     public void build() throws Exception {
@@ -76,6 +76,7 @@ public class BuildCubeWithStream2 extends BuildCubeWithStream {
 
         List<FutureTask<ExecutableState>> futures = Lists.newArrayList();
         for (int i = 0; i < 5; i++) {
+            Thread.sleep(2 * 60 * 1000); // sleep 2 mintues
             FutureTask futureTask = new FutureTask(new Callable<ExecutableState>() {
                 @Override
                 public ExecutableState call() {
@@ -92,7 +93,6 @@ public class BuildCubeWithStream2 extends BuildCubeWithStream {
 
             executorService.submit(futureTask);
             futures.add(futureTask);
-            Thread.sleep(2 * 60 * 1000); // sleep 2 mintues
         }
 
         generateData = false; // stop generating message to kafka

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
index 59a3a04..93d47f1 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/ITKylinQueryTest.java
@@ -24,6 +24,7 @@ import java.io.File;
 import java.sql.SQLException;
 import java.util.List;
 import java.util.Map;
+import java.util.TimeZone;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
@@ -65,6 +66,8 @@ public class ITKylinQueryTest extends KylinTestBase {
 
         RemoveBlackoutRealizationsRule.blackList.add("CUBE[name=test_kylin_cube_with_view_left_join_empty]");
         RemoveBlackoutRealizationsRule.blackList.add("CUBE[name=test_kylin_cube_with_view_inner_join_empty]");
+
+        TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
     }
 
     @AfterClass

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java b/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
index d0bcf52..57c4f4d 100644
--- a/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
+++ b/kylin-it/src/test/java/org/apache/kylin/query/KylinTestBase.java
@@ -586,7 +586,7 @@ public class KylinTestBase {
         //setup cube conn
         File olapTmp = OLAPSchemaFactory.createTempOLAPJson(ProjectInstance.DEFAULT_PROJECT_NAME, config);
         Properties props = new Properties();
-        props.setProperty(OLAPQuery.PROP_SCAN_THRESHOLD, "15001");
+        props.setProperty(OLAPQuery.PROP_SCAN_THRESHOLD, "20001");
         cubeConnection = DriverManager.getConnection("jdbc:calcite:model=" + olapTmp.getAbsolutePath(), props);
 
         //setup h2

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/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..729719a 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
@@ -118,7 +118,7 @@ public class KafkaMRInput implements IMRInput {
                 }
             }
             Text text = (Text) mapperInput;
-            ByteBuffer buffer = ByteBuffer.wrap(text.getBytes(), 0, text.getLength()).slice();
+            ByteBuffer buffer = ByteBuffer.wrap(text.getBytes(), 0, text.getLength());
             StreamingMessage streamingMessage = streamingParser.parse(buffer);
             return streamingMessage.getData().toArray(new String[streamingMessage.getData().size()]);
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ab5563a8/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 d3530f1..148ae25 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
@@ -47,7 +47,6 @@ public final class TimedJsonStreamParser extends StreamingParser {
     private static final Logger logger = LoggerFactory.getLogger(TimedJsonStreamParser.class);
 
     private List<TblColRef> allColumns;
-    private boolean formatTs = false;//not used
     private final ObjectMapper mapper = new ObjectMapper();
     private String tsColName = "timestamp";
     private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(String.class), SimpleType.construct(String.class));
@@ -61,9 +60,6 @@ public final class TimedJsonStreamParser extends StreamingParser {
                     String[] parts = prop.split("=");
                     if (parts.length == 2) {
                         switch (parts[0]) {
-                        case "formatTs":
-                            this.formatTs = Boolean.valueOf(parts[1]);
-                            break;
                         case "tsColName":
                             this.tsColName = parts[1];
                             break;
@@ -78,7 +74,7 @@ public final class TimedJsonStreamParser extends StreamingParser {
             }
         }
 
-        logger.info("TimedJsonStreamParser with formatTs {} tsColName {}", formatTs, tsColName);
+        logger.info("TimedJsonStreamParser with tsColName {}", tsColName);
     }
 
     @Override
@@ -105,7 +101,6 @@ public final class TimedJsonStreamParser extends StreamingParser {
                 }
             }
 
-            logger.info("Streaming Message: " + result.toString());
             return new StreamingMessage(result, 0, t, Collections.<String, Object> emptyMap());
         } catch (IOException e) {
             logger.error("error", e);


[06/28] kylin git commit: KYLIN-1565: Read KV max size from HBase config

Posted by sh...@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/KYLIN-1726-2
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;


[02/28] kylin git commit: KYLIN-1978: fix compatible issue on Ubuntu

Posted by sh...@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/KYLIN-1726-2
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


[26/28] kylin git commit: Revert "Revert "KYLIN-1726 add test case BuildCubeWithStream2""

Posted by sh...@apache.org.
Revert "Revert "KYLIN-1726 add test case BuildCubeWithStream2""

This reverts commit 96d5f0e0e639fe4e4fc169f687004d9d9361999b.


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

Branch: refs/heads/KYLIN-1726-2
Commit: b1a0f4dc93bd17a36a6c4573cf56fb9d285e4bed
Parents: f0de023
Author: shaofengshi <sh...@apache.org>
Authored: Sat Sep 24 14:58:15 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Tue Sep 27 10:17:40 2016 +0800

----------------------------------------------------------------------
 .../kylin/job/streaming/Kafka10DataLoader.java  |   4 -
 .../apache/kylin/common/KylinConfigBase.java    |   4 +
 .../java/org/apache/kylin/cube/CubeManager.java |  28 +-
 .../org/apache/kylin/job/dao/ExecutableDao.java |   1 +
 .../kylin/job/manager/ExecutableManager.java    |   2 +-
 .../streaming/cube/StreamingCubeBuilder.java    |   2 +-
 .../test_streaming_table_cube_desc.json         |   3 +-
 .../kylin/provision/BuildCubeWithStream.java    |  32 ++-
 .../kylin/provision/BuildCubeWithStream2.java   | 274 +++++++++++++++++++
 .../kylin/rest/controller/CubeController.java   |   8 +-
 .../apache/kylin/rest/service/JobService.java   |   4 +-
 .../kylin/source/kafka/SeekOffsetStep.java      |   7 +-
 12 files changed, 320 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java b/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
index a5132af..2b299cc 100644
--- a/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
+++ b/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
@@ -65,13 +65,9 @@ public class Kafka10DataLoader extends StreamDataLoader {
         props.put("retry.backoff.ms", "1000");
         KafkaProducer producer = KafkaClient.getKafkaProducer(brokerList, props);
 
-        int boundary = messages.size() / 10;
         for (int i = 0; i < messages.size(); ++i) {
             ProducerRecord<String, String> keyedMessage = new ProducerRecord<String, String>(clusterConfig.getTopic(), String.valueOf(i), messages.get(i));
             producer.send(keyedMessage);
-            if (i % boundary == 0) {
-                logger.info("sending " + i + " messages to " + this.toString());
-            }
         }
         logger.info("sent " + messages.size() + " messages to " + this.toString());
         producer.close();

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/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 914f726..3e41055 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
@@ -805,4 +805,8 @@ abstract public class KylinConfigBase implements Serializable {
     public String getCreateFlatHiveTableMethod() {
         return getOptional("kylin.hive.create.flat.table.method", "1");
     }
+
+    public int getMaxBuildingSegments() {
+        return Integer.parseInt(getOptional("kylin.cube.building.segment.max", "1"));
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/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 3a327f9..463c8e9 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
@@ -435,13 +435,8 @@ public class CubeManager implements IRealizationProvider {
     }
 
     public CubeSegment appendSegment(CubeInstance cube, long startDate, long endDate, long startOffset, long endOffset) throws IOException {
-        return appendSegment(cube, startDate, endDate, startOffset, endOffset, true);
-    }
-
-    public CubeSegment appendSegment(CubeInstance cube, long startDate, long endDate, long startOffset, long endOffset, boolean strictChecking) throws IOException {
 
-        if (strictChecking)
-            checkNoBuildingSegment(cube);
+        checkBuildingSegment(cube);
 
         if (cube.getDescriptor().getModel().getPartitionDesc().isPartitioned()) {
             // try figure out a reasonable start if missing
@@ -471,12 +466,9 @@ 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 {
-        checkNoBuildingSegment(cube);
+    public CubeSegment refreshSegment(CubeInstance cube, long startDate, long endDate, long startOffset, long endOffset) throws IOException {
+        checkBuildingSegment(cube);
 
         CubeSegment newSegment = newSegment(cube, startDate, endDate, startOffset, endOffset);
 
@@ -497,7 +489,7 @@ public class CubeManager implements IRealizationProvider {
         if (startDate >= endDate && startOffset >= endOffset)
             throw new IllegalArgumentException("Invalid merge range");
 
-        checkNoBuildingSegment(cube);
+        checkBuildingSegment(cube);
         checkCubeIsPartitioned(cube);
 
         boolean isOffsetsOn = cube.getSegments().get(0).isSourceOffsetsOn();
@@ -623,9 +615,10 @@ public class CubeManager implements IRealizationProvider {
         }
     }
 
-    private void checkNoBuildingSegment(CubeInstance cube) {
-        if (cube.getBuildingSegments().size() > 0) {
-            throw new IllegalStateException("There is already a building segment!");
+    private void checkBuildingSegment(CubeInstance cube) {
+        int maxBuldingSeg = cube.getConfig().getMaxBuildingSegments();
+        if (cube.getBuildingSegments().size() >= maxBuldingSeg) {
+            throw new IllegalStateException("There is already " + cube.getBuildingSegments().size() + " building segment; ");
         }
     }
 
@@ -764,8 +757,9 @@ public class CubeManager implements IRealizationProvider {
         }
 
         for (CubeSegment seg : tobe) {
-            if (isReady(seg) == false)
-                throw new IllegalStateException("For cube " + cube + ", segment " + seg + " should be READY but is not");
+            if (isReady(seg) == false) {
+                logger.warn("For cube " + cube + ", segment " + seg + " isn't READY yet.");
+            }
         }
 
         List<CubeSegment> toRemoveSegs = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
index 8808a56..5cae5ac 100644
--- a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
+++ b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
@@ -207,6 +207,7 @@ public class ExecutableDao {
     }
 
     public void updateJobOutput(ExecutableOutputPO output) throws PersistentException {
+        logger.debug("updating job output, id: " + output.getUuid());
         try {
             final long ts = writeJobOutputResource(pathOfJobOutput(output.getUuid()), output);
             output.setLastModified(ts);

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java b/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
index 3a19486..d42b924 100644
--- a/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
+++ b/core-job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
@@ -278,7 +278,7 @@ public class ExecutableManager {
             ExecutableState oldStatus = ExecutableState.valueOf(jobOutput.getStatus());
             if (newStatus != null && oldStatus != newStatus) {
                 if (!ExecutableState.isValidStateTransfer(oldStatus, newStatus)) {
-                    throw new IllegalStateTranferException("there is no valid state transfer from:" + oldStatus + " to:" + newStatus);
+                    throw new IllegalStateTranferException("there is no valid state transfer from:" + oldStatus + " to:" + newStatus + ", job id: " + jobId);
                 }
                 jobOutput.setStatus(newStatus.toString());
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/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 e071a72..07a4cfb 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
@@ -104,7 +104,7 @@ public class StreamingCubeBuilder implements StreamingBatchBuilder {
         CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
         final CubeInstance cubeInstance = cubeManager.reloadCubeLocal(cubeName);
         try {
-            CubeSegment segment = cubeManager.appendSegment(cubeInstance, streamingBatch.getTimeRange().getFirst(), streamingBatch.getTimeRange().getSecond(), 0, 0, false);
+            CubeSegment segment = cubeManager.appendSegment(cubeInstance, streamingBatch.getTimeRange().getFirst(), streamingBatch.getTimeRange().getSecond(), 0, 0);
             segment.setLastBuildJobID(segment.getUuid()); // give a fake job id
             segment.setInputRecords(streamingBatch.getMessages().size());
             segment.setLastBuildTime(System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json b/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
index ef10c1e..8279417 100644
--- a/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
+++ b/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
@@ -106,7 +106,8 @@
     }
   } ],
   "override_kylin_properties": {
-    "kylin.cube.algorithm": "inmem"
+    "kylin.cube.algorithm": "inmem",
+    "kylin.cube.building.segment.max": "3"
   },
   "notify_list" : [ ],
   "status_need_notify" : [ ],

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/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..b7c609e 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
@@ -137,15 +137,21 @@ public class BuildCubeWithStream {
 
         int numberOfRecrods1 = 10000;
         generateStreamData(date1, date2, numberOfRecrods1);
-        buildSegment(cubeName, 0, Long.MAX_VALUE);
-
+        ExecutableState result = buildSegment(cubeName, 0, Long.MAX_VALUE);
+        Assert.assertTrue(result == ExecutableState.SUCCEED);
         long date3 = f.parse("2013-04-01").getTime();
-        int numberOfRecrods2 = 5000;
-        generateStreamData(date2, date3, numberOfRecrods2);
-        buildSegment(cubeName, 0, Long.MAX_VALUE);
+        int numberOfRecords2 = 5000;
+        generateStreamData(date2, date3, numberOfRecords2);
+        result = buildSegment(cubeName, 0, Long.MAX_VALUE);
+        Assert.assertTrue(result == ExecutableState.SUCCEED);
+
+        //empty build
+        result = buildSegment(cubeName, 0, Long.MAX_VALUE);
+        Assert.assertTrue(result == ExecutableState.DISCARDED);
 
         //merge
-        mergeSegment(cubeName, 0, 15000);
+        result = mergeSegment(cubeName, 0, 15000);
+        Assert.assertTrue(result == ExecutableState.SUCCEED);
 
         List<CubeSegment> segments = cubeManager.getCube(cubeName).getSegments();
         Assert.assertTrue(segments.size() == 1);
@@ -159,16 +165,16 @@ public class BuildCubeWithStream {
 
     }
 
-    private String mergeSegment(String cubeName, long startOffset, long endOffset) throws Exception {
+    private ExecutableState mergeSegment(String cubeName, long startOffset, long endOffset) throws Exception {
         CubeSegment segment = cubeManager.mergeSegments(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, false);
         DefaultChainedExecutable job = EngineFactory.createBatchMergeJob(segment, "TEST");
         jobService.addJob(job);
         waitForJob(job.getId());
-        return job.getId();
+        return job.getStatus();
     }
 
     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);
+        CubeSegment segment = cubeManager.refreshSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset);
         segment.setAdditionalInfo(partitionOffsetMap);
         CubeInstance cubeInstance = cubeManager.getCube(cubeName);
         CubeUpdate cubeBuilder = new CubeUpdate(cubeInstance);
@@ -181,12 +187,12 @@ public class BuildCubeWithStream {
         return job.getId();
     }
 
-    private String buildSegment(String cubeName, long startOffset, long endOffset) throws Exception {
-        CubeSegment segment = cubeManager.appendSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, endOffset, false);
+    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.getId();
+        return job.getStatus();
     }
 
     protected void deployEnv() throws IOException {
@@ -216,7 +222,7 @@ public class BuildCubeWithStream {
     protected void waitForJob(String jobId) {
         while (true) {
             AbstractExecutable job = jobService.getJob(jobId);
-            if (job.getStatus() == ExecutableState.SUCCEED || job.getStatus() == ExecutableState.ERROR) {
+            if (job.getStatus() == ExecutableState.SUCCEED || job.getStatus() == ExecutableState.ERROR || job.getStatus() == ExecutableState.DISCARDED) {
                 break;
             } else {
                 try {

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/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
new file mode 100644
index 0000000..d48a473
--- /dev/null
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
@@ -0,0 +1,274 @@
+/*
+ * 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 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;
+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;
+
+import static java.lang.Thread.sleep;
+
+/**
+ *  for streaming cubing case "test_streaming_table", using multiple threads to build it concurrently.
+ */
+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;
+
+    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");
+        f.setTimeZone(TimeZone.getTimeZone("GMT"));
+        final long date1 = 0;
+        final long date2 = f.parse("2013-01-01").getTime();
+
+        new Thread(new Runnable() {
+            @Override
+            public void run() {
+
+                Random rand = new Random();
+                while (generateData == true) {
+                    try {
+                        generateStreamData(date1, date2, rand.nextInt(100));
+                        sleep(rand.nextInt(rand.nextInt(100 * 1000))); // wait random time, from 0 to 100 seconds
+                    } catch (IOException e) {
+                        e.printStackTrace();
+                    } catch (InterruptedException e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+        }).start();
+        ExecutorService executorService = Executors.newFixedThreadPool(4);
+
+        List<FutureTask<ExecutableState>> futures = Lists.newArrayList();
+        for (int i = 0; i < 5; i++) {
+            FutureTask futureTask = new FutureTask(new Callable<ExecutableState>() {
+                @Override
+                public ExecutableState call() {
+                    ExecutableState result = null;
+                    try {
+                        result = buildSegment(cubeName, 0, Long.MAX_VALUE);
+                    } catch (Exception e) {
+                        e.printStackTrace();
+                    }
+
+                    return result;
+                }
+            });
+
+            executorService.submit(futureTask);
+            futures.add(futureTask);
+            Thread.sleep(2 * 60 * 1000); // sleep 2 mintues
+        }
+
+        generateData = false; // stop generating message to kafka
+        executorService.shutdown();
+        int succeedBuild = 0;
+        for (int i = 0; i < futures.size(); i++) {
+            ExecutableState result = futures.get(i).get(20, TimeUnit.MINUTES);
+            logger.info("Checking building task " + i + " whose state is " + result);
+            Assert.assertTrue(result == null || result == ExecutableState.SUCCEED || result == ExecutableState.DISCARDED );
+            if (result == ExecutableState.SUCCEED)
+                succeedBuild++;
+        }
+
+        logger.info(succeedBuild + " build jobs have been successfully completed.");
+        List<CubeSegment> segments = cubeManager.getCube(cubeName).getSegments(SegmentStatusEnum.READY);
+        Assert.assertTrue(segments.size() == succeedBuild);
+
+    }
+
+
+    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();
+    }
+
+    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();
+
+            BuildCubeWithStream2 buildCubeWithStream = new BuildCubeWithStream2();
+            buildCubeWithStream.before();
+            buildCubeWithStream.build();
+            logger.info("Build is done");
+            buildCubeWithStream.after();
+            afterClass();
+            logger.info("Going to exit");
+            System.exit(0);
+        } catch (Exception e) {
+            logger.error("error", e);
+            System.exit(1);
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a0f4dc/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/b1a0f4dc/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/b1a0f4dc/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..9369e6f 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;
@@ -125,7 +120,7 @@ public class SeekOffsetStep extends AbstractExecutable {
             } catch (IOException e) {
                 return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
             }
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed, offset start: " + totalStartOffset + ", offset end: " + totalEndOffset);
         } else {
             CubeUpdate cubeBuilder = new CubeUpdate(cube);
             cubeBuilder.setToRemoveSegs(segment);