You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by qh...@apache.org on 2015/09/25 04:48:11 UTC

[4/4] incubator-kylin git commit: KYLIN-1011 decompose project streaming

KYLIN-1011
decompose project streaming


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

Branch: refs/heads/2.x-staging
Commit: 15e1752ee51912ce03d3b593e6f066e11ecaefb2
Parents: 1ea5870
Author: qianhao.zhou <qi...@ebay.com>
Authored: Wed Sep 9 19:13:01 2015 +0800
Committer: qianhao.zhou <qi...@ebay.com>
Committed: Fri Sep 25 10:47:43 2015 +0800

----------------------------------------------------------------------
 assembly/pom.xml                                |   5 -
 .../kylin/job/BuildCubeWithStreamTest.java      |  37 +-
 .../apache/kylin/job/BuildIIWithStreamTest.java |  81 ++--
 .../java/org/apache/kylin/job/DeployUtil.java   |  34 +-
 .../job/ITKafkaBasedIIStreamBuilderTest.java    |  85 ----
 .../kylin/job/hadoop/invertedindex/IITest.java  |  39 +-
 .../job/streaming/CubeStreamConsumerTest.java   |  90 -----
 .../kylin/job/streaming/KafkaDataLoader.java    |  53 +++
 .../streaming/PeriodicalStreamBuilderTest.java  | 144 -------
 build/bin/kylin.sh                              |   4 +-
 build/script/compress.sh                        |   1 +
 .../kylin/common/persistence/ResourceStore.java |   1 +
 .../kylin/engine/streaming/StreamingCLI.java    |  99 -----
 .../kylin/engine/streaming/cli/MonitorCLI.java  |  70 ++++
 .../engine/streaming/cli/StreamingCLI.java      | 120 ++++++
 .../streaming/monitor/StreamingMonitor.java     | 154 +++++++
 .../engine/streaming/util/StreamingUtils.java   |   2 +-
 .../localmeta/kafka/kafka_test.json             |  20 +
 .../kafka/test_streaming_table_cube.json        |  22 +
 .../kafka/test_streaming_table_ii.json          |  22 +
 .../streaming/test_streaming_table_cube.json    |  18 +-
 .../streaming/test_streaming_table_ii.json      |  18 +-
 .../invertedindex/streaming/SliceBuilder.java   |  81 ++++
 .../kafka/ByteBufferBackedInputStream.java      |  53 +++
 .../kylin/source/kafka/KafkaConfigManager.java  |  94 +----
 .../kylin/source/kafka/KafkaStreamingInput.java |   8 +-
 .../source/kafka/StringStreamingParser.java     |  65 +++
 .../source/kafka/TimedJsonStreamParser.java     | 142 +++++++
 .../kylin/source/kafka/config/KafkaConfig.java  |  12 +
 .../hbase/ii/ITInvertedIndexHBaseTest.java      |   2 +-
 .../.settings/org.eclipse.core.resources.prefs  |   5 -
 streaming/.settings/org.eclipse.jdt.core.prefs  |   5 -
 streaming/pom.xml                               |  70 ----
 .../apache/kylin/job/monitor/MonitorCLI.java    |  69 ----
 .../kylin/job/monitor/StreamingMonitor.java     | 154 -------
 .../kylin/job/streaming/BootstrapConfig.java    |  74 ----
 .../kylin/job/streaming/CubeStreamConsumer.java | 141 -------
 .../kylin/job/streaming/KafkaDataLoader.java    |  54 ---
 .../kylin/job/streaming/StreamingBootstrap.java | 402 -------------------
 .../kylin/job/streaming/StreamingCLI.java       | 104 -----
 .../apache/kylin/streaming/BatchCondition.java  |  13 -
 .../apache/kylin/streaming/BrokerConfig.java    |  77 ----
 .../kylin/streaming/JsonStreamParser.java       |  88 ----
 .../kylin/streaming/KafkaClusterConfig.java     |  60 ---
 .../apache/kylin/streaming/KafkaConsumer.java   | 182 ---------
 .../apache/kylin/streaming/KafkaRequester.java  | 180 ---------
 .../kylin/streaming/LimitedSizeCondition.java   |  29 --
 .../kylin/streaming/MarginCalculator.java       |  83 ----
 .../kylin/streaming/MicroStreamBatch.java       |  99 -----
 .../streaming/MicroStreamBatchConsumer.java     |  11 -
 .../kylin/streaming/OffsetPeriodCondition.java  |  35 --
 .../kylin/streaming/OneOffStreamBuilder.java    |  87 ----
 .../kylin/streaming/ParsedStreamMessage.java    |  42 --
 .../apache/kylin/streaming/PartitionMargin.java |  13 -
 .../streaming/PartitionMarginCalculator.java    | 104 -----
 .../apache/kylin/streaming/StreamBuilder.java   | 202 ----------
 .../apache/kylin/streaming/StreamFetcher.java   | 109 -----
 .../apache/kylin/streaming/StreamMessage.java   |  58 ---
 .../apache/kylin/streaming/StreamParser.java    |  46 ---
 .../apache/kylin/streaming/StreamingConfig.java | 189 ---------
 .../kylin/streaming/StreamingManager.java       | 221 ----------
 .../apache/kylin/streaming/StreamingUtil.java   | 164 --------
 .../kylin/streaming/StringStreamParser.java     |  52 ---
 .../kylin/streaming/TimePeriodCondition.java    |  43 --
 .../kylin/streaming/TimedJsonStreamParser.java  | 139 -------
 .../org/apache/kylin/streaming/TopicMeta.java   |  62 ---
 .../invertedindex/IIStreamConsumer.java         | 149 -------
 .../streaming/invertedindex/SliceBuilder.java   |  75 ----
 .../streaming/util/StreamingInputAnalyzer.java  | 242 -----------
 .../streaming/util/StreamingLogsAnalyser.java   |  96 -----
 .../kylin/streaming/util/TimeHistogram.java     |  85 ----
 .../kylin/streaming/ITKafkaConsumerTest.java    | 112 ------
 .../kylin/streaming/ITKafkaRequesterTest.java   |  80 ----
 .../apache/kylin/streaming/KafkaBaseTest.java   |  53 ---
 .../kylin/streaming/OneOffStreamProducer.java   | 119 ------
 .../kylin/streaming/StreamingManagerTest.java   | 129 ------
 76 files changed, 947 insertions(+), 5235 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 99557fb..9f17913 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -49,11 +49,6 @@
             <artifactId>kylin-invertedindex</artifactId>
             <version>${project.parent.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-streaming</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
         
         <!-- Env & Test -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
index b02b2f2..c0e7978 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildCubeWithStreamTest.java
@@ -34,21 +34,18 @@
 
 package org.apache.kylin.job;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.UUID;
-
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.job.streaming.BootstrapConfig;
-import org.apache.kylin.job.streaming.StreamingBootstrap;
+import org.apache.kylin.engine.streaming.OneOffStreamingBuilder;
+import org.apache.kylin.engine.streaming.StreamingConfig;
+import org.apache.kylin.engine.streaming.StreamingManager;
+import org.apache.kylin.source.kafka.KafkaConfigManager;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;
 import org.apache.kylin.storage.hbase.util.StorageCleanupJob;
-import org.apache.kylin.streaming.StreamingConfig;
-import org.apache.kylin.streaming.StreamingManager;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -56,6 +53,10 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+
 /**
  *  for streaming cubing case "test_streaming_table"
  */
@@ -84,12 +85,14 @@ public class BuildCubeWithStreamTest {
 
         kylinConfig = KylinConfig.getInstanceFromEnv();
 
-        //Use a random toplic for kafka data stream
-        StreamingConfig streamingConfig = StreamingManager.getInstance(kylinConfig).getStreamingConfig(streamingName);
+        final StreamingConfig config = StreamingManager.getInstance(KylinConfig.getInstanceFromEnv()).getStreamingConfig(streamingName);
+
+        //Use a random topic for kafka data stream
+        KafkaConfig streamingConfig = KafkaConfigManager.getInstance(kylinConfig).getKafkaConfig(streamingName);
         streamingConfig.setTopic(UUID.randomUUID().toString());
-        StreamingManager.getInstance(kylinConfig).saveStreamingConfig(streamingConfig);
+        KafkaConfigManager.getInstance(kylinConfig).saveStreamingConfig(streamingConfig);
 
-        DeployUtil.prepareTestDataForStreamingCube(startTime, endTime, streamingConfig);
+        DeployUtil.prepareTestDataForStreamingCube(startTime, endTime, config.getCubeName(), streamingConfig);
     }
 
     @AfterClass
@@ -119,14 +122,10 @@ public class BuildCubeWithStreamTest {
 
     @Test
     public void test() 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) {
-            BootstrapConfig bootstrapConfig = new BootstrapConfig();
-            bootstrapConfig.setStart(start);
-            bootstrapConfig.setEnd(start + batchInterval);
-            bootstrapConfig.setOneOff(true);
-            bootstrapConfig.setPartitionId(0);
-            bootstrapConfig.setStreaming(streamingName);
-            StreamingBootstrap.getInstance(KylinConfig.getInstanceFromEnv()).start(bootstrapConfig);
+            logger.info(String.format("build batch:{%d, %d}", start, start + batchInterval));
+            new OneOffStreamingBuilder(streamingName, start, start + batchInterval).build().run();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
index 5ca3b29..256297e 100644
--- a/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/BuildIIWithStreamTest.java
@@ -39,28 +39,34 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.IOException;
 import java.text.SimpleDateFormat;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.TimeZone;
-import java.util.UUID;
+import java.util.*;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingDeque;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.engine.mr.JobBuilderSupport;
+import org.apache.kylin.engine.streaming.StreamingBatch;
+import org.apache.kylin.engine.streaming.StreamingMessage;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.invertedindex.index.Slice;
 import org.apache.kylin.invertedindex.model.IIDesc;
 import org.apache.kylin.invertedindex.model.IIJoinedFlatTableDesc;
+import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
+import org.apache.kylin.invertedindex.model.IIRow;
+import org.apache.kylin.invertedindex.streaming.SliceBuilder;
 import org.apache.kylin.job.common.ShellExecutable;
 import org.apache.kylin.job.constant.ExecutableConstants;
 import org.apache.kylin.job.engine.JobEngineConfig;
@@ -68,10 +74,8 @@ import org.apache.kylin.job.hadoop.invertedindex.IICreateHTableJob;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.RealizationStatusEnum;
 import org.apache.kylin.source.hive.HiveTableReader;
+import org.apache.kylin.storage.hbase.HBaseConnection;
 import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;
-import org.apache.kylin.streaming.StreamBuilder;
-import org.apache.kylin.streaming.StreamMessage;
-import org.apache.kylin.streaming.invertedindex.IIStreamConsumer;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -184,32 +188,32 @@ public class BuildIIWithStreamTest {
                 logger.info("measure:" + tblColRef.getName());
             }
         }
-        LinkedBlockingDeque<StreamMessage> queue = new LinkedBlockingDeque<StreamMessage>();
         final IISegment segment = createSegment(iiName);
+        final HTableInterface htable = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getTable(segment.getStorageLocationIdentifier());
         String[] args = new String[] { "-iiname", iiName, "-htablename", segment.getStorageLocationIdentifier() };
         ToolRunner.run(new IICreateHTableJob(), args);
 
-        ExecutorService executorService = Executors.newSingleThreadExecutor();
-        final StreamBuilder streamBuilder = StreamBuilder.newLimitedSizeStreamBuilder(iiName, queue, new IIStreamConsumer(iiName, segment.getStorageLocationIdentifier(), segment.getIIDesc(), 0), 0, segment.getIIDesc().getSliceSize());
+        final IIDesc iiDesc = segment.getIIDesc();
+        final SliceBuilder sliceBuilder = new SliceBuilder(desc, (short) 0, iiDesc.isUseLocalDictionary());
 
         List<String[]> sorted = getSortedRows(reader, desc.getTimestampColumn());
         int count = sorted.size();
+        ArrayList<StreamingMessage> messages = Lists.newArrayList();
         for (String[] row : sorted) {
-            logger.info("another row: " + StringUtils.join(row, ","));
-            queue.put(parse(row));
+            if (messages.size() < iiDesc.getSliceSize()) {
+                messages.add(parse(row));
+            } else {
+                build(sliceBuilder, new StreamingBatch(messages, Pair.newPair(System.currentTimeMillis(), System.currentTimeMillis())), htable);
+                messages = Lists.newArrayList();
+                messages.add((parse(row)));
+            }
+        }
+        if (!messages.isEmpty()) {
+            build(sliceBuilder, new StreamingBatch(messages, Pair.newPair(System.currentTimeMillis(), System.currentTimeMillis())), htable);
         }
 
         reader.close();
         logger.info("total record count:" + count + " htable:" + segment.getStorageLocationIdentifier());
-        queue.put(StreamMessage.EOF);
-        final Future<?> future = executorService.submit(streamBuilder);
-        try {
-            future.get();
-        } catch (Exception e) {
-            logger.error("stream build failed", e);
-            fail("stream build failed");
-        }
-
         logger.info("stream build finished, htable name:" + segment.getStorageLocationIdentifier());
     }
 
@@ -224,9 +228,38 @@ public class BuildIIWithStreamTest {
             }
         }
     }
+    
+    private void build(SliceBuilder sliceBuilder, StreamingBatch batch, HTableInterface htable) {
+        final Slice slice = sliceBuilder.buildSlice(batch);
+        try {
+            loadToHBase(htable, slice, new IIKeyValueCodec(slice.getInfo()));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+    
+    private void loadToHBase(HTableInterface hTable, Slice slice, IIKeyValueCodec codec) throws IOException {
+        List<Put> data = Lists.newArrayList();
+        for (IIRow row : codec.encodeKeyValue(slice)) {
+            final byte[] key = row.getKey().get();
+            final byte[] value = row.getValue().get();
+            Put put = new Put(key);
+            put.add(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES, value);
+            final ImmutableBytesWritable dictionary = row.getDictionary();
+            final byte[] dictBytes = dictionary.get();
+            if (dictionary.getOffset() == 0 && dictionary.getLength() == dictBytes.length) {
+                put.add(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_DICTIONARY_BYTES, dictBytes);
+            } else {
+                throw new RuntimeException("dict offset should be 0, and dict length should be " + dictBytes.length + " but they are" + dictionary.getOffset() + " " + dictionary.getLength());
+            }
+            data.add(put);
+        }
+        hTable.put(data);
+        //omit hTable.flushCommits(), because htable is auto flush
+    }
 
-    private StreamMessage parse(String[] row) {
-        return new StreamMessage(System.currentTimeMillis(), StringUtils.join(row, ",").getBytes());
+    private StreamingMessage parse(String[] row) {
+        return new StreamingMessage(Lists.newArrayList(row), System.currentTimeMillis(), System.currentTimeMillis(), Collections.<String, Object>emptyMap());
     }
 
     private List<String[]> getSortedRows(HiveTableReader reader, final int tsCol) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/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 d47a664..722b6b8 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -18,14 +18,9 @@
 
 package org.apache.kylin.job;
 
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.List;
-
+import com.google.common.collect.Lists;
+import kafka.message.Message;
+import kafka.message.MessageAndOffset;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.kylin.common.KylinConfig;
@@ -35,6 +30,7 @@ import org.apache.kylin.common.util.AbstractKylinTestCase;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeUpdate;
+import org.apache.kylin.engine.streaming.StreamingConfig;
 import org.apache.kylin.job.dataGen.FactTableGenerator;
 import org.apache.kylin.job.streaming.KafkaDataLoader;
 import org.apache.kylin.job.streaming.StreamingTableDataGenerator;
@@ -43,15 +39,16 @@ import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.source.hive.HiveClient;
-import org.apache.kylin.streaming.StreamMessage;
-import org.apache.kylin.streaming.StreamingConfig;
-import org.apache.kylin.streaming.TimedJsonStreamParser;
+import org.apache.kylin.source.kafka.KafkaConfigManager;
+import org.apache.kylin.source.kafka.TimedJsonStreamParser;
+import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.apache.maven.model.Model;
 import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
+import java.io.*;
+import java.util.List;
 
 public class DeployUtil {
     private static final Logger logger = LoggerFactory.getLogger(DeployUtil.class);
@@ -146,14 +143,15 @@ public class DeployUtil {
         deployHiveTables();
     }
 
-    public static void prepareTestDataForStreamingCube(long startTime, long endTime, StreamingConfig streamingConfig) throws IOException {
-        CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(streamingConfig.getCubeName());
+    public static void prepareTestDataForStreamingCube(long startTime, long endTime, String cubeName, KafkaConfig kafkaConfig) 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());
         TableDesc tableDesc = cubeInstance.getFactTableDesc();
-
         //load into kafka
-        KafkaDataLoader.loadIntoKafka(streamingConfig, data);
-        logger.info("Write {} messages into topic {}", data.size(), streamingConfig.getTopic());
+        KafkaDataLoader.loadIntoKafka(kafkaConfig.getKafkaClusterConfigs(), data);
+        KafkaDataLoader.loadIntoKafka(kafkaConfig.getKafkaClusterConfigs(), data2);
+        logger.info("Write {} messages into topic {}", data.size(), kafkaConfig.getTopic());
 
         //csv data for H2 use
         List<TblColRef> tableColumns = Lists.newArrayList();
@@ -163,7 +161,7 @@ public class DeployUtil {
         TimedJsonStreamParser timedJsonStreamParser = new TimedJsonStreamParser(tableColumns, "formatTs=true");
         StringBuilder sb = new StringBuilder();
         for (String json : data) {
-            List<String> rowColumns = timedJsonStreamParser.parse(new StreamMessage(0, json.getBytes())).getStreamMessage();
+            List<String> rowColumns = timedJsonStreamParser.parse(new MessageAndOffset(new Message(json.getBytes()), 0)).getData();
             sb.append(StringUtils.join(rowColumns, ","));
             sb.append(System.getProperty("line.separator"));
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/assembly/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java b/assembly/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java
deleted file mode 100644
index 6a615cb..0000000
--- a/assembly/src/test/java/org/apache/kylin/job/ITKafkaBasedIIStreamBuilderTest.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- *
- *
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *
- *  contributor license agreements. See the NOTICE file distributed with
- *
- *  this work for additional information regarding copyright ownership.
- *
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *
- *  (the "License"); you may not use this file except in compliance with
- *
- *  the License. You may obtain a copy of the License at
- *
- *
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- *
- *
- *  Unless required by applicable law or agreed to in writing, software
- *
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *
- *  See the License for the specific language governing permissions and
- *
- *  limitations under the License.
- *
- * /
- */
-
-package org.apache.kylin.job;
-
-import java.io.File;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.AbstractKylinTestCase;
-import org.apache.kylin.common.util.ClassUtil;
-import org.apache.kylin.job.streaming.StreamingBootstrap;
-import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- */
-@Ignore("this test case will break existing metadata store")
-public class ITKafkaBasedIIStreamBuilderTest {
-
-    private static final Logger logger = LoggerFactory.getLogger(ITKafkaBasedIIStreamBuilderTest.class);
-
-    private KylinConfig kylinConfig;
-
-    @BeforeClass
-    public static void beforeClass() throws Exception {
-        ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
-        System.setProperty("hdp.version", "2.2.4.2-2"); // mapred-site.xml ref this
-    }
-
-    @Before
-    public void before() throws Exception {
-        HBaseMetadataTestCase.staticCreateTestMetadata(AbstractKylinTestCase.SANDBOX_TEST_DATA);
-
-        kylinConfig = KylinConfig.getInstanceFromEnv();
-        DeployUtil.initCliWorkDir();
-        DeployUtil.deployMetadata();
-        DeployUtil.overrideJobJarLocations();
-    }
-
-    @Test
-    public void test() throws Exception {
-        final StreamingBootstrap bootstrap = StreamingBootstrap.getInstance(kylinConfig);
-        bootstrap.start("eagle", 0);
-        Thread.sleep(30 * 60 * 1000);
-        logger.info("time is up, stop streaming");
-        bootstrap.stop();
-        Thread.sleep(5 * 1000);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java b/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java
index dcd460b..913a2f7 100644
--- a/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java
+++ b/assembly/src/test/java/org/apache/kylin/job/hadoop/invertedindex/IITest.java
@@ -10,12 +10,18 @@ import java.util.Set;
 
 import javax.annotation.Nullable;
 
+import com.google.common.base.Function;
+import kafka.message.Message;
+import kafka.message.MessageAndOffset;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.common.util.FIFOIterable;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.engine.streaming.StreamingBatch;
+import org.apache.kylin.engine.streaming.StreamingMessage;
 import org.apache.kylin.invertedindex.IIInstance;
 import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.invertedindex.index.Slice;
@@ -26,6 +32,7 @@ import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
 import org.apache.kylin.invertedindex.model.IIKeyValueCodecWithState;
 import org.apache.kylin.invertedindex.model.IIRow;
 import org.apache.kylin.invertedindex.model.KeyValueCodec;
+import org.apache.kylin.invertedindex.streaming.SliceBuilder;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.ConstantTupleFilter;
@@ -33,6 +40,8 @@ import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.source.kafka.StreamingParser;
+import org.apache.kylin.source.kafka.StringStreamingParser;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorFilter;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorProjector;
 import org.apache.kylin.storage.hbase.common.coprocessor.CoprocessorRowType;
@@ -41,18 +50,11 @@ import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.ClearTextDictionar
 import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.EndpointAggregators;
 import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.IIEndpoint;
 import org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.generated.IIProtos;
-import org.apache.kylin.streaming.MicroStreamBatch;
-import org.apache.kylin.streaming.ParsedStreamMessage;
-import org.apache.kylin.streaming.StreamMessage;
-import org.apache.kylin.streaming.StreamParser;
-import org.apache.kylin.streaming.StringStreamParser;
-import org.apache.kylin.streaming.invertedindex.SliceBuilder;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
@@ -78,24 +80,23 @@ public class IITest extends LocalFileMetadataTestCase {
         this.ii = IIManager.getInstance(getTestConfig()).getII(iiName);
         this.iiDesc = ii.getDescriptor();
 
-        List<StreamMessage> streamMessages = Lists.transform(Arrays.asList(inputData), new Function<String, StreamMessage>() {
+        List<MessageAndOffset> messages = Lists.transform(Arrays.asList(inputData), new Function<String, MessageAndOffset>() {
             @Nullable
             @Override
-            public StreamMessage apply(String input) {
-                return new StreamMessage(System.currentTimeMillis(), input.getBytes());
+            public MessageAndOffset apply(String input) {
+                return new MessageAndOffset(new Message(input.getBytes()), System.currentTimeMillis());
             }
         });
 
-        List<List<String>> parsedStreamMessages = Lists.newArrayList();
-        StreamParser parser = StringStreamParser.instance;
-
-        MicroStreamBatch batch = new MicroStreamBatch(0);
-        for (StreamMessage message : streamMessages) {
-            ParsedStreamMessage parsedStreamMessage = parser.parse(message);
-            if ((parsedStreamMessage.isAccepted())) {
-                batch.add(parsedStreamMessage);
+        final StreamingParser parser = StringStreamingParser.instance;
+        final List<StreamingMessage> streamingMessages = Lists.transform(messages, new Function<MessageAndOffset, StreamingMessage>() {
+            @Nullable
+            @Override
+            public StreamingMessage apply(@Nullable MessageAndOffset input) {
+                return parser.parse(input);
             }
-        }
+        });
+        StreamingBatch batch = new StreamingBatch(streamingMessages, Pair.newPair(0L, System.currentTimeMillis()));
 
         iiRows = Lists.newArrayList();
         final Slice slice = new SliceBuilder(iiDesc, (short) 0, true).buildSlice((batch));

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/assembly/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java b/assembly/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java
deleted file mode 100644
index be4fa26..0000000
--- a/assembly/src/test/java/org/apache/kylin/job/streaming/CubeStreamConsumerTest.java
+++ /dev/null
@@ -1,90 +0,0 @@
-package org.apache.kylin.job.streaming;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingDeque;
-
-import org.apache.hadoop.util.StringUtils;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.AbstractKylinTestCase;
-import org.apache.kylin.common.util.ClassUtil;
-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.job.DeployUtil;
-import org.apache.kylin.storage.hbase.steps.HBaseMetadataTestCase;
-import org.apache.kylin.streaming.StreamBuilder;
-import org.apache.kylin.streaming.StreamMessage;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-@Ignore
-public class CubeStreamConsumerTest {
-
-    private static final Logger logger = LoggerFactory.getLogger(CubeStreamConsumerTest.class);
-
-    private KylinConfig kylinConfig;
-
-    private static final String CUBE_NAME = "test_kylin_cube_without_slr_left_join_ready";
-
-    @BeforeClass
-    public static void beforeClass() throws Exception {
-        ClassUtil.addClasspath(new File(HBaseMetadataTestCase.SANDBOX_TEST_DATA).getAbsolutePath());
-        System.setProperty("hdp.version", "2.2.0.0-2041"); // mapred-site.xml ref this
-    }
-
-    @Before
-    public void before() throws Exception {
-        HBaseMetadataTestCase.staticCreateTestMetadata(AbstractKylinTestCase.SANDBOX_TEST_DATA);
-
-        kylinConfig = KylinConfig.getInstanceFromEnv();
-        DeployUtil.initCliWorkDir();
-        DeployUtil.deployMetadata();
-        DeployUtil.overrideJobJarLocations();
-        final CubeInstance cube = CubeManager.getInstance(kylinConfig).getCube(CUBE_NAME);
-        CubeUpdate cubeBuilder = new CubeUpdate(cube);
-        cubeBuilder.setToRemoveSegs(cube.getSegments().toArray(new CubeSegment[cube.getSegments().size()]));
-        // remove all existing segments
-        CubeManager.getInstance(kylinConfig).updateCube(cubeBuilder);
-
-    }
-
-    @Test
-    public void test() throws Exception {
-        LinkedBlockingDeque<StreamMessage> queue = new LinkedBlockingDeque<>();
-        List<BlockingQueue<StreamMessage>> queues = Lists.newArrayList();
-        queues.add(queue);
-        StreamBuilder cubeStreamBuilder = StreamBuilder.newPeriodicalStreamBuilder(CUBE_NAME, queues, new CubeStreamConsumer(CUBE_NAME), System.currentTimeMillis(), 30L * 1000);
-        final Future<?> future = Executors.newSingleThreadExecutor().submit(cubeStreamBuilder);
-        loadDataFromLocalFile(queue, 100000);
-        future.get();
-    }
-
-    private void loadDataFromLocalFile(BlockingQueue<StreamMessage> queue, final int maxCount) throws IOException, InterruptedException {
-        BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream("../table.txt")));
-        String line;
-        int count = 0;
-        while ((line = br.readLine()) != null && count++ < maxCount) {
-            final List<String> strings = Arrays.asList(line.split("\t"));
-            queue.put(new StreamMessage(System.currentTimeMillis(), StringUtils.join(",", strings).getBytes()));
-        }
-        queue.put(StreamMessage.EOF);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/assembly/src/test/java/org/apache/kylin/job/streaming/KafkaDataLoader.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/streaming/KafkaDataLoader.java b/assembly/src/test/java/org/apache/kylin/job/streaming/KafkaDataLoader.java
new file mode 100644
index 0000000..c3caa9b
--- /dev/null
+++ b/assembly/src/test/java/org/apache/kylin/job/streaming/KafkaDataLoader.java
@@ -0,0 +1,53 @@
+package org.apache.kylin.job.streaming;
+
+import java.util.List;
+import java.util.Properties;
+
+import javax.annotation.Nullable;
+
+import kafka.javaapi.producer.Producer;
+import kafka.producer.KeyedMessage;
+import kafka.producer.ProducerConfig;
+
+import org.apache.commons.lang.StringUtils;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Lists;
+import org.apache.kylin.source.kafka.config.BrokerConfig;
+import org.apache.kylin.source.kafka.config.KafkaClusterConfig;
+
+/**
+ * Load prepared data into kafka(for test use)
+ */
+public class KafkaDataLoader {
+
+    public static void loadIntoKafka(List<KafkaClusterConfig> kafkaClusterConfigs, 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("metadata.broker.list", brokerList);
+        props.put("serializer.class", "kafka.serializer.StringEncoder");
+        props.put("request.required.acks", "1");
+
+        ProducerConfig config = new ProducerConfig(props);
+
+        Producer<String, String> producer = new Producer<String, String>(config);
+
+        List<KeyedMessage<String, String>> keyedMessages = Lists.newArrayList();
+        for (int i = 0; i < messages.size(); ++i) {
+            KeyedMessage<String, String> keyedMessage = new KeyedMessage<String, String>(clusterConfig.getTopic(), String.valueOf(i), messages.get(i));
+            keyedMessages.add(keyedMessage);
+        }
+        producer.send(keyedMessages);
+        producer.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/assembly/src/test/java/org/apache/kylin/job/streaming/PeriodicalStreamBuilderTest.java
----------------------------------------------------------------------
diff --git a/assembly/src/test/java/org/apache/kylin/job/streaming/PeriodicalStreamBuilderTest.java b/assembly/src/test/java/org/apache/kylin/job/streaming/PeriodicalStreamBuilderTest.java
deleted file mode 100644
index dc6d312..0000000
--- a/assembly/src/test/java/org/apache/kylin/job/streaming/PeriodicalStreamBuilderTest.java
+++ /dev/null
@@ -1,144 +0,0 @@
-package org.apache.kylin.job.streaming;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.common.util.TimeUtil;
-import org.apache.kylin.streaming.MicroStreamBatch;
-import org.apache.kylin.streaming.MicroStreamBatchConsumer;
-import org.apache.kylin.streaming.ParsedStreamMessage;
-import org.apache.kylin.streaming.StreamBuilder;
-import org.apache.kylin.streaming.StreamMessage;
-import org.apache.kylin.streaming.StreamParser;
-import org.apache.kylin.streaming.StreamingManager;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- */
-public class PeriodicalStreamBuilderTest extends LocalFileMetadataTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(PeriodicalStreamBuilderTest.class);
-
-    @Before
-    public void setup() {
-        this.createTestMetadata();
-
-    }
-
-    @After
-    public void clear() {
-        this.cleanupTestMetadata();
-    }
-
-    private List<StreamMessage> prepareTestData(long start, long end, int count) {
-        double step = (double) (end - start) / (count - 1);
-        long ts = start;
-        int offset = 0;
-        ArrayList<StreamMessage> result = Lists.newArrayList();
-        for (int i = 0; i < count - 1; ++i) {
-            result.add(new StreamMessage(offset++, String.valueOf(ts).getBytes()));
-            ts += step;
-        }
-        result.add(new StreamMessage(offset++, String.valueOf(end).getBytes()));
-        assertEquals(count, result.size());
-        assertEquals(start + "", new String(result.get(0).getRawData()));
-        assertEquals(end + "", new String(result.get(count - 1).getRawData()));
-        return result;
-    }
-
-    @Test
-    public void test() throws ExecutionException, InterruptedException {
-
-        List<BlockingQueue<StreamMessage>> queues = Lists.newArrayList();
-        queues.add(new LinkedBlockingQueue<StreamMessage>());
-        queues.add(new LinkedBlockingQueue<StreamMessage>());
-
-        final long interval = 3000L;
-        final long nextPeriodStart = TimeUtil.getNextPeriodStart(System.currentTimeMillis(), interval);
-
-        final List<Integer> partitionIds = Lists.newArrayList();
-        for (int i = 0; i < queues.size(); i++) {
-            partitionIds.add(i);
-        }
-
-        final MicroStreamBatchConsumer consumer = new MicroStreamBatchConsumer() {
-            @Override
-            public void consume(MicroStreamBatch microStreamBatch) throws Exception {
-                logger.info("consuming batch:" + microStreamBatch.getPartitionId() + " count:" + microStreamBatch.size() + " timestamp:" + microStreamBatch.getTimestamp() + " offset:" + microStreamBatch.getOffset());
-            }
-
-            @Override
-            public void stop() {
-                logger.info("consumer stopped");
-            }
-        };
-        final StreamBuilder streamBuilder = StreamBuilder.newPeriodicalStreamBuilder("test", queues, consumer, nextPeriodStart, interval);
-
-        streamBuilder.setStreamParser(new StreamParser() {
-            @Override
-            public ParsedStreamMessage parse(StreamMessage streamMessage) {
-                return new ParsedStreamMessage(Collections.<String> emptyList(), streamMessage.getOffset(), Long.parseLong(new String(streamMessage.getRawData())), true);
-            }
-        });
-
-        Future<?> future = Executors.newSingleThreadExecutor().submit(streamBuilder);
-        long timeout = nextPeriodStart + interval;
-        int messageCount = 0;
-        int inPeriodMessageCount = 0;
-        int expectedOffset = 0;
-        logger.info("prepare to add StreamMessage");
-        while (true) {
-            long ts = System.currentTimeMillis();
-            if (ts >= timeout + interval) {
-                break;
-            }
-            if (ts >= nextPeriodStart && ts < timeout) {
-                inPeriodMessageCount++;
-            }
-            for (BlockingQueue<StreamMessage> queue : queues) {
-                queue.put(new StreamMessage(messageCount, String.valueOf(ts).getBytes()));
-            }
-            if (expectedOffset == 0 && ts >= timeout) {
-                expectedOffset = messageCount - 1;
-            }
-            messageCount++;
-            Thread.sleep(10);
-        }
-        logger.info("totally put " + messageCount + " StreamMessages");
-        logger.info("totally in period " + inPeriodMessageCount + " StreamMessages");
-
-        for (BlockingQueue<StreamMessage> queue : queues) {
-            queue.put(StreamMessage.EOF);
-        }
-
-        future.get();
-
-        for (BlockingQueue<StreamMessage> queue : queues) {
-            queue.take();
-        }
-
-        final Map<Integer, Long> offsets = StreamingManager.getInstance(KylinConfig.getInstanceFromEnv()).getOffset("test", partitionIds);
-        logger.info("offset:" + offsets);
-        for (Long offset : offsets.values()) {
-            assertEquals(expectedOffset, offset.longValue());
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/build/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/build/bin/kylin.sh b/build/bin/kylin.sh
index b27864c..b581e09 100644
--- a/build/bin/kylin.sh
+++ b/build/bin/kylin.sh
@@ -115,7 +115,7 @@ then
         -Dkylin.hive.dependency=${hive_dependency} \
         -Dkylin.hbase.dependency=${hbase_dependency} \
         -Dspring.profiles.active=${spring_profile} \
-        org.apache.kylin.job.streaming.StreamingCLI $@ > ${KYLIN_HOME}/logs/streaming_$3_$4.log 2>&1 & echo $! > ${KYLIN_HOME}/logs/$3_$4 &
+        org.apache.kylin.engine.streaming.cli.StreamingCLI $@ > ${KYLIN_HOME}/logs/streaming_$3_$4.log 2>&1 & echo $! > ${KYLIN_HOME}/logs/$3_$4 &
         echo "streaming started name: $3 id: $4"
         exit 0
     elif [ $2 == "stop" ]
@@ -170,7 +170,7 @@ then
     -Dkylin.hive.dependency=${hive_dependency} \
     -Dkylin.hbase.dependency=${hbase_dependency} \
     -Dspring.profiles.active=${spring_profile} \
-    org.apache.kylin.job.monitor.MonitorCLI $@ >> ${KYLIN_HOME}/logs/monitor.log 2>&1
+    org.apache.kylin.engine.streaming.cli.MonitorCLI $@ > ${KYLIN_HOME}/logs/monitor.log 2>&1
     exit 0
 else
     echo "usage: kylin.sh start or kylin.sh stop"

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/build/script/compress.sh
----------------------------------------------------------------------
diff --git a/build/script/compress.sh b/build/script/compress.sh
index a424b98..c70e567 100755
--- a/build/script/compress.sh
+++ b/build/script/compress.sh
@@ -21,6 +21,7 @@ rm -rf lib tomcat commit_SHA1
 find kylin-${version} -type d -exec chmod 755 {} \;
 find kylin-${version} -type f -exec chmod 644 {} \;
 find kylin-${version} -type f -name "*.sh" -exec chmod 755 {} \;
+mkdir -p ../dist
 tar -cvzf ../dist/kylin-${version}.tar.gz kylin-${version}
 rm -rf kylin-${version}
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index 29e2345..89100a2 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -54,6 +54,7 @@ abstract public class ResourceStore {
     public static final String TABLE_RESOURCE_ROOT = "/table";
     public static final String HYBRID_RESOURCE_ROOT = "/hybrid";
     public static final String STREAMING_RESOURCE_ROOT = "/streaming";
+    public static final String KAfKA_RESOURCE_ROOT = "/kafka";
     public static final String STREAMING_OUTPUT_RESOURCE_ROOT = "/streaming_output";
     public static final String CUBE_STATISTICS_ROOT = "/cube_statistics";
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingCLI.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingCLI.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingCLI.java
deleted file mode 100644
index 8bf52c1..0000000
--- a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/StreamingCLI.java
+++ /dev/null
@@ -1,99 +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.engine.streaming;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kylin.common.cache.RemoteCacheUpdater;
-import org.apache.kylin.common.restclient.AbstractRestCache;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Preconditions;
-
-public class StreamingCLI {
-
-    private static final Logger logger = LoggerFactory.getLogger(StreamingCLI.class);
-
-    public static void main(String[] args) {
-        try {
-            AbstractRestCache.setCacheUpdater(new RemoteCacheUpdater());
-
-            Preconditions.checkArgument(args[0].equals("streaming"));
-            Preconditions.checkArgument(args[1].equals("start"));
-
-            int i = 2;
-            BootstrapConfig bootstrapConfig = new BootstrapConfig();
-            while (i < args.length) {
-                String argName = args[i];
-                switch (argName) {
-                case "-oneoff":
-                    bootstrapConfig.setOneOff(Boolean.parseBoolean(args[++i]));
-                    break;
-                case "-start":
-                    bootstrapConfig.setStart(Long.parseLong(args[++i]));
-                    break;
-                case "-end":
-                    bootstrapConfig.setEnd(Long.parseLong(args[++i]));
-                    break;
-                case "-streaming":
-                    bootstrapConfig.setStreaming(args[++i]);
-                    break;
-                case "-partition":
-                    bootstrapConfig.setPartitionId(Integer.parseInt(args[++i]));
-                    break;
-                case "-fillGap":
-                    bootstrapConfig.setFillGap(Boolean.parseBoolean(args[++i]));
-                    break;
-                default:
-                    logger.warn("ignore this arg:" + argName);
-                }
-                i++;
-            }
-            final Runnable runnable = new OneOffStreamingBuilder(bootstrapConfig.getStreaming(), bootstrapConfig.getStart(), bootstrapConfig.getEnd()).build();
-            runnable.run();
-            logger.info("streaming process stop, exit with 0");
-            System.exit(0);
-        } catch (Exception e) {
-            printArgsError(args);
-            logger.error("error start streaming", e);
-            System.exit(-1);
-        }
-    }
-
-    private static void printArgsError(String[] args) {
-        logger.warn("invalid args:" + StringUtils.join(args, " "));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/MonitorCLI.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/MonitorCLI.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/MonitorCLI.java
new file mode 100644
index 0000000..d7dc6b3
--- /dev/null
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/MonitorCLI.java
@@ -0,0 +1,70 @@
+package org.apache.kylin.engine.streaming.cli;
+
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.engine.streaming.monitor.StreamingMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+/**
+ */
+public class MonitorCLI {
+
+    private static final Logger logger = LoggerFactory.getLogger(MonitorCLI.class);
+
+    public static void main(String[] args) {
+        Preconditions.checkArgument(args[0].equals("monitor"));
+
+        int i = 1;
+        List<String> receivers = null;
+        String host = null;
+        String tableName = null;
+        String authorization = null;
+        String cubeName = null;
+        String projectName = "default";
+        while (i < args.length) {
+            String argName = args[i];
+            switch (argName) {
+            case "-receivers":
+                receivers = Lists.newArrayList(StringUtils.split(args[++i], ";"));
+                break;
+            case "-host":
+                host = args[++i];
+                break;
+            case "-tableName":
+                tableName = args[++i];
+                break;
+            case "-authorization":
+                authorization = args[++i];
+                break;
+            case "-cubeName":
+                cubeName = args[++i];
+                break;
+            case "-projectName":
+                projectName = args[++i];
+                break;
+            default:
+                throw new RuntimeException("invalid argName:" + argName);
+            }
+            i++;
+        }
+        Preconditions.checkArgument(receivers != null && receivers.size() > 0);
+        final StreamingMonitor streamingMonitor = new StreamingMonitor();
+        if (tableName != null) {
+            logger.info(String.format("check query tableName:%s host:%s receivers:%s", tableName, host, StringUtils.join(receivers, ";")));
+            Preconditions.checkNotNull(host);
+            Preconditions.checkNotNull(authorization);
+            Preconditions.checkNotNull(tableName);
+            streamingMonitor.checkCountAll(receivers, host, authorization, projectName, tableName);
+        }
+        if (cubeName != null) {
+            logger.info(String.format("check cube cubeName:%s receivers:%s", cubeName, StringUtils.join(receivers, ";")));
+            streamingMonitor.checkCube(receivers, cubeName,host);
+        }
+        System.exit(0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/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
new file mode 100644
index 0000000..a4ccabc
--- /dev/null
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/cli/StreamingCLI.java
@@ -0,0 +1,120 @@
+/*
+ *
+ *
+ *  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;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.cache.RemoteCacheUpdater;
+import org.apache.kylin.common.restclient.AbstractRestCache;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.engine.streaming.BootstrapConfig;
+import org.apache.kylin.engine.streaming.OneOffStreamingBuilder;
+import org.apache.kylin.engine.streaming.StreamingConfig;
+import org.apache.kylin.engine.streaming.StreamingManager;
+import org.apache.kylin.engine.streaming.monitor.StreamingMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class StreamingCLI {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamingCLI.class);
+
+    public static void main(String[] args) {
+        try {
+            AbstractRestCache.setCacheUpdater(new RemoteCacheUpdater());
+
+            Preconditions.checkArgument(args[0].equals("streaming"));
+            Preconditions.checkArgument(args[1].equals("start"));
+
+            int i = 2;
+            BootstrapConfig bootstrapConfig = new BootstrapConfig();
+            while (i < args.length) {
+                String argName = args[i];
+                switch (argName) {
+                case "-oneoff":
+                    Boolean.parseBoolean(args[++i]);
+                    break;
+                case "-start":
+                    bootstrapConfig.setStart(Long.parseLong(args[++i]));
+                    break;
+                case "-end":
+                    bootstrapConfig.setEnd(Long.parseLong(args[++i]));
+                    break;
+                case "-streaming":
+                    bootstrapConfig.setStreaming(args[++i]);
+                    break;
+                case "-partition":
+                    bootstrapConfig.setPartitionId(Integer.parseInt(args[++i]));
+                    break;
+                case "-fillGap":
+                    bootstrapConfig.setFillGap(Boolean.parseBoolean(args[++i]));
+                    break;
+                default:
+                    logger.warn("ignore this arg:" + argName);
+                }
+                i++;
+            }
+            if (bootstrapConfig.isFillGap()) {
+                final StreamingConfig streamingConfig = StreamingManager.getInstance(KylinConfig.getInstanceFromEnv()).getStreamingConfig(bootstrapConfig.getStreaming());
+                final List<Pair<Long, Long>> gaps = StreamingMonitor.findGaps(streamingConfig.getCubeName());
+                logger.info("all gaps:" + StringUtils.join(gaps, ","));
+                for (Pair<Long, Long> gap : gaps) {
+                    startOneOffCubeStreaming(bootstrapConfig.getStreaming(), gap.getFirst(), gap.getSecond());
+                }
+            } else {
+                startOneOffCubeStreaming(bootstrapConfig.getStreaming(), bootstrapConfig.getStart(), bootstrapConfig.getEnd());
+                logger.info("streaming process finished, exit with 0");
+                System.exit(0);
+            }
+        } catch (Exception e) {
+            printArgsError(args);
+            logger.error("error start streaming", e);
+            System.exit(-1);
+        }
+    }
+    
+    private static void startOneOffCubeStreaming(String streaming, long start, long end) {
+        final Runnable runnable = new OneOffStreamingBuilder(streaming, start, end).build();
+        runnable.run();
+    }
+
+    private static void printArgsError(String[] args) {
+        logger.warn("invalid args:" + StringUtils.join(args, " "));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/monitor/StreamingMonitor.java
----------------------------------------------------------------------
diff --git a/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/monitor/StreamingMonitor.java b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/monitor/StreamingMonitor.java
new file mode 100644
index 0000000..a6b8a9f
--- /dev/null
+++ b/engine-streaming/src/main/java/org/apache/kylin/engine/streaming/monitor/StreamingMonitor.java
@@ -0,0 +1,154 @@
+package org.apache.kylin.engine.streaming.monitor;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+
+import javax.annotation.Nullable;
+
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.methods.ByteArrayRequestEntity;
+import org.apache.commons.httpclient.methods.PostMethod;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.JsonUtil;
+import org.apache.kylin.common.util.MailService;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+/**
+ */
+public class StreamingMonitor {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamingMonitor.class);
+
+    public void checkCountAll(List<String> receivers, String host, String authorization, String projectName, String tableName) {
+        String title = "checkCountAll job(host:" + host + " tableName:" + tableName + ") ";
+        StringBuilder stringBuilder = new StringBuilder();
+        String url = host + "/kylin/api/query";
+        PostMethod request = new PostMethod(url);
+        try {
+
+            request.addRequestHeader("Authorization", "Basic " + authorization);
+            request.addRequestHeader("Content-Type", "application/json");
+            String query = String.format("{\"sql\":\"select count(*) from %s\",\"offset\":0,\"limit\":50000,\"acceptPartial\":true,\"project\":\"%s\"}", tableName, projectName);
+            request.setRequestEntity(new ByteArrayRequestEntity(query.getBytes()));
+
+            int statusCode = new HttpClient().executeMethod(request);
+            String msg = Bytes.toString(request.getResponseBody());
+            stringBuilder.append("host:").append(host).append("\n");
+            stringBuilder.append("query:").append(query).append("\n");
+            stringBuilder.append("statusCode:").append(statusCode).append("\n");
+            if (statusCode == 200) {
+                title += "succeed";
+                final HashMap hashMap = JsonUtil.readValue(msg, HashMap.class);
+                stringBuilder.append("results:").append(hashMap.get("results").toString()).append("\n");
+                stringBuilder.append("duration:").append(hashMap.get("duration").toString()).append("\n");
+            } else {
+                title += "failed";
+                stringBuilder.append("response:").append(msg).append("\n");
+            }
+        } catch (Exception e) {
+            final StringWriter out = new StringWriter();
+            e.printStackTrace(new PrintWriter(out));
+            title += "failed";
+            stringBuilder.append(out.toString());
+        } finally {
+            request.releaseConnection();
+        }
+        logger.info("title:" + title);
+        logger.info("content:" + stringBuilder.toString());
+        sendMail(receivers, title, stringBuilder.toString());
+    }
+
+    public static final List<Pair<Long, Long>> findGaps(String cubeName) {
+        List<CubeSegment> segments = getSortedReadySegments(cubeName);
+        List<Pair<Long, Long>> gaps = Lists.newArrayList();
+        for (int i = 0; i < segments.size() - 1; ++i) {
+            CubeSegment first = segments.get(i);
+            CubeSegment second = segments.get(i + 1);
+            if (first.getDateRangeEnd() == second.getDateRangeStart()) {
+                continue;
+            } else if (first.getDateRangeEnd() < second.getDateRangeStart()) {
+                gaps.add(Pair.newPair(first.getDateRangeEnd(), second.getDateRangeStart()));
+            }
+        }
+        return gaps;
+    }
+
+    private static List<CubeSegment> getSortedReadySegments(String cubeName) {
+        final CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).reloadCubeLocal(cubeName);
+        Preconditions.checkNotNull(cube);
+        final List<CubeSegment> segments = cube.getSegment(SegmentStatusEnum.READY);
+        logger.info("totally " + segments.size() + " cubeSegments");
+        Collections.sort(segments);
+        return segments;
+    }
+
+    public static final List<Pair<String, String>> findOverlaps(String cubeName) {
+        List<CubeSegment> segments = getSortedReadySegments(cubeName);
+        List<Pair<String, String>> overlaps = Lists.newArrayList();
+        for (int i = 0; i < segments.size() - 1; ++i) {
+            CubeSegment first = segments.get(i);
+            CubeSegment second = segments.get(i + 1);
+            if (first.getDateRangeEnd() == second.getDateRangeStart()) {
+                continue;
+            } else {
+                overlaps.add(Pair.newPair(first.getName(), second.getName()));
+            }
+        }
+        return overlaps;
+    }
+
+    public void checkCube(List<String> receivers, String cubeName, String host) {
+        final CubeInstance cube = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).reloadCubeLocal(cubeName);
+        if (cube == null) {
+            logger.info("cube:" + cubeName + " does not exist");
+            return;
+        }
+        List<Pair<Long, Long>> gaps = findGaps(cubeName);
+        List<Pair<String, String>> overlaps = Lists.newArrayList();
+        StringBuilder content = new StringBuilder();
+        if (!gaps.isEmpty()) {
+            content.append("all gaps:").append("\n").append(StringUtils.join(Lists.transform(gaps, new Function<Pair<Long, Long>, String>() {
+                @Nullable
+                @Override
+                public String apply(Pair<Long, Long> input) {
+                    return parseInterval(input);
+                }
+            }), "\n")).append("\n");
+        }
+        if (!overlaps.isEmpty()) {
+            content.append("all overlaps:").append("\n").append(StringUtils.join(overlaps, "\n")).append("\n");
+        }
+        if (content.length() > 0) {
+            logger.info(content.toString());
+            sendMail(receivers, String.format("%s has gaps or overlaps on host %s", cubeName, host), content.toString());
+        } else {
+            logger.info("no gaps or overlaps");
+        }
+    }
+
+    private String parseInterval(Pair<Long, Long> interval) {
+        return String.format("{%d(%s), %d(%s)}", interval.getFirst(), new Date(interval.getFirst()).toString(), interval.getSecond(), new Date(interval.getSecond()).toString());
+    }
+
+    private void sendMail(List<String> receivers, String title, String content) {
+        final MailService mailService = new MailService(KylinConfig.getInstanceFromEnv());
+        mailService.sendMail(receivers, title, content, false);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/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 718fc43..47db924 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
@@ -53,7 +53,7 @@ public class StreamingUtils {
     }
 
     public static IStreamingOutput getStreamingOutput(String streaming) {
-        return (IStreamingOutput) ClassUtil.newInstance("org.apache.kylin.storage.hbase.HBaseStreamingOutput");
+        return (IStreamingOutput) ClassUtil.newInstance("org.apache.kylin.storage.hbase.steps.HBaseStreamingOutput");
     }
 
     public static StreamingBatchBuilder getMicroBatchBuilder(String streaming) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/examples/test_case_data/localmeta/kafka/kafka_test.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kafka/kafka_test.json b/examples/test_case_data/localmeta/kafka/kafka_test.json
new file mode 100644
index 0000000..5445417
--- /dev/null
+++ b/examples/test_case_data/localmeta/kafka/kafka_test.json
@@ -0,0 +1,20 @@
+{
+  "uuid": "8b2b9dfe-900c-4d39-bf89-8472ec6c3c0d",
+  "name": "kafka_test",
+  "topic": "kafka_stream_test",
+  "timeout": 60000,
+  "maxReadCount": 1000,
+  "bufferSize": 65536,
+  "last_modified": 0,
+  "clusters": [
+    {
+      "brokers": [
+        {
+          "id": 0,
+          "host": "sandbox.hortonworks.com",
+          "port": 6667
+        }
+      ]
+    }
+  ]
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/examples/test_case_data/localmeta/kafka/test_streaming_table_ii.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kafka/test_streaming_table_ii.json b/examples/test_case_data/localmeta/kafka/test_streaming_table_ii.json
new file mode 100644
index 0000000..9e36201
--- /dev/null
+++ b/examples/test_case_data/localmeta/kafka/test_streaming_table_ii.json
@@ -0,0 +1,22 @@
+{
+  "uuid": "8b2b9dfe-900c-4d39-bf89-8472ec909322",
+  "name": "test_streaming_table_ii",
+  "topic": "test_streaming_table_topic_xyz",
+  "timeout": 60000,
+  "maxReadCount": 1000,
+  "bufferSize": 65536,
+  "parserName": "org.apache.kylin.source.kafka.JsonStreamParser",
+  "partition": 1,
+  "last_modified": 0,
+  "clusters": [
+    {
+      "brokers": [
+        {
+          "id": 0,
+          "host": "sandbox",
+          "port": 6667
+        }
+      ]
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/examples/test_case_data/localmeta/streaming/test_streaming_table_cube.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/streaming/test_streaming_table_cube.json b/examples/test_case_data/localmeta/streaming/test_streaming_table_cube.json
index 98b4218..b358183 100644
--- a/examples/test_case_data/localmeta/streaming/test_streaming_table_cube.json
+++ b/examples/test_case_data/localmeta/streaming/test_streaming_table_cube.json
@@ -1,23 +1,7 @@
 {
   "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
   "name": "test_streaming_table_cube",
-  "topic": "test_streaming_table_topic_xyz",
-  "timeout": 60000,
-  "maxReadCount": 1000,
-  "bufferSize": 65536,
   "cubeName": "test_streaming_table_cube",
-  "parserName": "org.apache.kylin.streaming.TimedJsonStreamParser",
   "partition": 1,
-  "last_modified": 0,
-  "clusters": [
-    {
-      "brokers": [
-        {
-          "id": 0,
-          "host": "sandbox",
-          "port": 6667
-        }
-      ]
-    }
-  ]
+  "last_modified": 0
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/examples/test_case_data/localmeta/streaming/test_streaming_table_ii.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/streaming/test_streaming_table_ii.json b/examples/test_case_data/localmeta/streaming/test_streaming_table_ii.json
index e70153e..da1a765 100644
--- a/examples/test_case_data/localmeta/streaming/test_streaming_table_ii.json
+++ b/examples/test_case_data/localmeta/streaming/test_streaming_table_ii.json
@@ -1,23 +1,7 @@
 {
   "uuid": "8b2b9dfe-900c-4d39-bf89-8472ec909322",
   "name": "test_streaming_table_ii",
-  "topic": "test_streaming_table_topic_xyz",
-  "timeout": 60000,
-  "maxReadCount": 1000,
-  "bufferSize": 65536,
   "iiName": "test_streaming_table_ii",
-  "parserName": "org.apache.kylin.streaming.JsonStreamParser",
   "partition": 1,
-  "last_modified": 0,
-  "clusters": [
-    {
-      "brokers": [
-        {
-          "id": 0,
-          "host": "sandbox",
-          "port": 6667
-        }
-      ]
-    }
-  ]
+  "last_modified": 0
 }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/invertedindex/src/main/java/org/apache/kylin/invertedindex/streaming/SliceBuilder.java
----------------------------------------------------------------------
diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/streaming/SliceBuilder.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/streaming/SliceBuilder.java
new file mode 100644
index 0000000..ba337c8
--- /dev/null
+++ b/invertedindex/src/main/java/org/apache/kylin/invertedindex/streaming/SliceBuilder.java
@@ -0,0 +1,81 @@
+/*
+ * 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.invertedindex.streaming;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.apache.kylin.dict.Dictionary;
+import org.apache.kylin.engine.streaming.StreamingBatch;
+import org.apache.kylin.engine.streaming.StreamingMessage;
+import org.apache.kylin.invertedindex.index.BatchSliceMaker;
+import org.apache.kylin.invertedindex.index.Slice;
+import org.apache.kylin.invertedindex.index.TableRecord;
+import org.apache.kylin.invertedindex.index.TableRecordInfo;
+import org.apache.kylin.invertedindex.model.IIDesc;
+import org.apache.kylin.invertedindex.util.IIDictionaryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.util.List;
+
+/**
+ */
+public final class SliceBuilder {
+
+    private static Logger logger = LoggerFactory.getLogger(SliceBuilder.class);
+
+    private final BatchSliceMaker sliceMaker;
+    private final IIDesc iiDesc;
+    private final boolean useLocalDict;
+
+    public SliceBuilder(IIDesc desc, short shard, boolean useLocalDict) {
+        this.iiDesc = desc;
+        this.sliceMaker = new BatchSliceMaker(desc, shard);
+        this.useLocalDict = useLocalDict;
+    }
+
+    public Slice buildSlice(StreamingBatch microStreamBatch) {
+        final List<List<String>> messages = Lists.transform(microStreamBatch.getMessages(), new Function<StreamingMessage, List<String>>() {
+            @Nullable
+            @Override
+            public List<String> apply(@Nullable StreamingMessage input) {
+                return input.getData();
+            }
+        });
+        final Dictionary<?>[] dictionaries = useLocalDict ? IIDictionaryBuilder.buildDictionary(messages, iiDesc) : new Dictionary[iiDesc.listAllColumns().size()];
+        TableRecordInfo tableRecordInfo = new TableRecordInfo(iiDesc, dictionaries);
+        return build(messages, tableRecordInfo, dictionaries);
+    }
+
+    private Slice build(List<List<String>> table, final TableRecordInfo tableRecordInfo, Dictionary<?>[] localDictionary) {
+        final Slice slice = sliceMaker.makeSlice(tableRecordInfo.getDigest(), Lists.transform(table, new Function<List<String>, TableRecord>() {
+            @Nullable
+            @Override
+            public TableRecord apply(@Nullable List<String> input) {
+                TableRecord result = tableRecordInfo.createTableRecord();
+                for (int i = 0; i < input.size(); i++) {
+                    result.setValueString(i, input.get(i));
+                }
+                return result;
+            }
+        }));
+        slice.setLocalDictionaries(localDictionary);
+        return slice;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/source-kafka/src/main/java/org/apache/kylin/source/kafka/ByteBufferBackedInputStream.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/ByteBufferBackedInputStream.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/ByteBufferBackedInputStream.java
new file mode 100644
index 0000000..5883493
--- /dev/null
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/ByteBufferBackedInputStream.java
@@ -0,0 +1,53 @@
+/*
+ * 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.io.InputStream;
+import java.nio.ByteBuffer;
+
+/**
+ */
+class ByteBufferBackedInputStream extends InputStream {
+
+    private ByteBuffer buf;
+
+    public ByteBufferBackedInputStream(ByteBuffer buf) {
+        this.buf = buf;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (!buf.hasRemaining()) {
+            return -1;
+        }
+        return buf.get() & 0xFF;
+    }
+
+    @Override
+    public int read(byte[] bytes, int off, int len)
+            throws IOException {
+        if (!buf.hasRemaining()) {
+            return -1;
+        }
+
+        len = Math.min(len, buf.remaining());
+        buf.get(bytes, off, len);
+        return len;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/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 c84035f..be07290 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
@@ -34,18 +34,10 @@
 
 package org.apache.kylin.source.kafka;
 
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
+import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.type.MapType;
+import com.fasterxml.jackson.databind.type.SimpleType;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.ResourceStore;
@@ -53,12 +45,9 @@ import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.fasterxml.jackson.databind.JavaType;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.type.MapType;
-import com.fasterxml.jackson.databind.type.SimpleType;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
  */
@@ -104,15 +93,7 @@ public class KafkaConfigManager {
     }
 
     private String formatStreamingConfigPath(String name) {
-        return ResourceStore.STREAMING_RESOURCE_ROOT + "/" + name + ".json";
-    }
-
-    private String formatStreamingOutputPath(String streaming, int partition) {
-        return ResourceStore.STREAMING_OUTPUT_RESOURCE_ROOT + "/" + streaming + "_" + partition + ".json";
-    }
-
-    private String formatStreamingOutputPath(String streaming, List<Integer> partitions) {
-        return ResourceStore.STREAMING_OUTPUT_RESOURCE_ROOT + "/" + streaming + "_" + StringUtils.join(partitions, "_") + ".json";
+        return ResourceStore.KAfKA_RESOURCE_ROOT + "/" + name + ".json";
     }
 
     public boolean createOrUpdateKafkaConfig(String name, KafkaConfig config) {
@@ -125,7 +106,7 @@ public class KafkaConfigManager {
         }
     }
 
-    public KafkaConfig getStreamingConfig(String name) {
+    public KafkaConfig getKafkaConfig(String name) {
         try {
             return getStore().getResource(formatStreamingConfigPath(name), KafkaConfig.class, KafkaConfig.SERIALIZER);
         } catch (IOException e) {
@@ -143,63 +124,6 @@ public class KafkaConfigManager {
         getStore().putResource(path, kafkaConfig, KafkaConfig.SERIALIZER);
     }
 
-    public long getOffset(String streaming, int shard) {
-        final String resPath = formatStreamingOutputPath(streaming, shard);
-        try {
-            final InputStream inputStream = getStore().getResource(resPath);
-            if (inputStream == null) {
-                return 0;
-            } else {
-                final BufferedReader br = new BufferedReader(new InputStreamReader(inputStream));
-                return Long.parseLong(br.readLine());
-            }
-        } catch (Exception e) {
-            logger.error("error get offset, path:" + resPath, e);
-            throw new RuntimeException("error get offset, path:" + resPath, e);
-        }
-    }
-
-    public void updateOffset(String streaming, int shard, long offset) {
-        Preconditions.checkArgument(offset >= 0, "offset cannot be smaller than 0");
-        final String resPath = formatStreamingOutputPath(streaming, shard);
-        try {
-            getStore().putResource(resPath, new ByteArrayInputStream(Long.valueOf(offset).toString().getBytes()), getStore().getResourceTimestamp(resPath));
-        } catch (IOException e) {
-            logger.error("error update offset, path:" + resPath, e);
-            throw new RuntimeException("error update offset, path:" + resPath, e);
-        }
-    }
-
-    public Map<Integer, Long> getOffset(String streaming, List<Integer> partitions) {
-        Collections.sort(partitions);
-        final String resPath = formatStreamingOutputPath(streaming, partitions);
-        try {
-            final InputStream inputStream = getStore().getResource(resPath);
-            if (inputStream == null) {
-                return Collections.emptyMap();
-            }
-            final HashMap<Integer, Long> result = mapper.readValue(inputStream, mapType);
-            return result;
-        } catch (IOException e) {
-            logger.error("error get offset, path:" + resPath, e);
-            throw new RuntimeException("error get offset, path:" + resPath, e);
-        }
-    }
-
-    public void updateOffset(String streaming, HashMap<Integer, Long> offset) {
-        List<Integer> partitions = Lists.newLinkedList(offset.keySet());
-        Collections.sort(partitions);
-        final String resPath = formatStreamingOutputPath(streaming, partitions);
-        try {
-            final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-            mapper.writeValue(baos, offset);
-            getStore().putResource(resPath, new ByteArrayInputStream(baos.toByteArray()), getStore().getResourceTimestamp(resPath));
-        } catch (IOException e) {
-            logger.error("error update offset, path:" + resPath, e);
-            throw new RuntimeException("error update offset, path:" + resPath, e);
-        }
-    }
-
     private final ObjectMapper mapper = new ObjectMapper();
     private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(Integer.class), SimpleType.construct(Long.class));
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/15e1752e/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 393b8e7..9951f86 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
@@ -79,9 +79,10 @@ public class KafkaStreamingInput implements IStreamingInput {
     @Override
     public StreamingBatch getBatchWithTimeWindow(String streaming, int id, long startTime, long endTime) {
         try {
+            logger.info(String.format("prepare to get streaming batch, name:%s, id:%d, startTime:%d, endTime:%d", streaming, id, startTime, endTime));
             final KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
             final KafkaConfigManager kafkaConfigManager = KafkaConfigManager.getInstance(kylinConfig);
-            final KafkaConfig kafkaConfig = kafkaConfigManager.getStreamingConfig(streaming);
+            final KafkaConfig kafkaConfig = kafkaConfigManager.getKafkaConfig(streaming);
             final StreamingParser streamingParser = getStreamingParser(kafkaConfig);
             final ExecutorService executorService = Executors.newCachedThreadPool();
             final List<Future<List<StreamingMessage>>> futures = Lists.newArrayList();
@@ -106,6 +107,7 @@ public class KafkaStreamingInput implements IStreamingInput {
                 }
             }
             final Pair<Long, Long> timeRange = Pair.newPair(startTime, endTime);
+            logger.info("finish to get streaming batch, total message count:" + messages.size());
             return new StreamingBatch(messages, timeRange);
         } catch (ReflectiveOperationException e) {
             throw new RuntimeException("failed to create instance of StreamingParser", e);
@@ -220,8 +222,8 @@ public class KafkaStreamingInput implements IStreamingInput {
         });
         if (!StringUtils.isEmpty(kafkaConfig.getParserName())) {
             Class clazz = Class.forName(kafkaConfig.getParserName());
-            Constructor constructor = clazz.getConstructor(List.class);
-            return (StreamingParser) constructor.newInstance(columns);
+            Constructor constructor = clazz.getConstructor(List.class, String.class);
+            return (StreamingParser) constructor.newInstance(columns, kafkaConfig.getParserProperties());
         } else {
             throw new IllegalStateException("invalid StreamingConfig:" + kafkaConfig.getName() + " missing property StreamingParser");
         }