You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ma...@apache.org on 2016/09/20 03:43:48 UTC

[01/13] kylin git commit: Revert "KYLIN-1726 Scalable streaming cubing"

Repository: kylin
Updated Branches:
  refs/heads/master 20ac92a32 -> 506cd7831


http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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 ab8b161..20c57a9 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,20 +1,37 @@
 /*
- * 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.io.IOException;

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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 d84d3db..4145ef6 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,20 +1,37 @@
 /*
- * 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.Collections;

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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
deleted file mode 100644
index bb64bf9..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
+++ /dev/null
@@ -1,108 +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.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/506cd783/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 95349c2..04a66f6 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,7 +22,6 @@ 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;
@@ -68,7 +67,7 @@ public class KafkaClusterConfig extends RootPersistentEntity {
             @Nullable
             @Override
             public Broker apply(BrokerConfig input) {
-                return new Broker(input.getId(), input.getHost(), input.getPort(), SecurityProtocol.PLAINTEXT);
+                return new Broker(input.getId(), input.getHost(), input.getPort());
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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
deleted file mode 100644
index decfb60..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableJob.java
+++ /dev/null
@@ -1,165 +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.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/506cd783/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
deleted file mode 100644
index 995b2d4..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaFlatTableMapper.java
+++ /dev/null
@@ -1,51 +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.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/506cd783/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
deleted file mode 100644
index 81f6bac..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputFormat.java
+++ /dev/null
@@ -1,98 +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.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/506cd783/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
deleted file mode 100644
index f67fef5..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputRecordReader.java
+++ /dev/null
@@ -1,166 +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.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/506cd783/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
deleted file mode 100644
index 3261399..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/hadoop/KafkaInputSplit.java
+++ /dev/null
@@ -1,102 +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.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/506cd783/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
deleted file mode 100644
index 640cc53..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaClient.java
+++ /dev/null
@@ -1,115 +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.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/506cd783/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
deleted file mode 100644
index b46e57f..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/util/KafkaOffsetMapping.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-package org.apache.kylin.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/506cd783/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 ddc2eb7..58cba7d 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,20 +1,37 @@
 /*
- * 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;
 
 import java.util.Collections;
@@ -25,8 +42,6 @@ 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;
@@ -71,14 +86,13 @@ public final class KafkaRequester {
         if (consumerCache.containsKey(key)) {
             return consumerCache.get(key);
         } else {
-            BrokerEndPoint brokerEndPoint = broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT);
-            consumerCache.putIfAbsent(key, new SimpleConsumer(brokerEndPoint.host(), brokerEndPoint.port(), timeout, bufferSize, clientId));
+            consumerCache.putIfAbsent(key, new SimpleConsumer(broker.host(), broker.port(), timeout, bufferSize, clientId));
             return consumerCache.get(key);
         }
     }
 
     private static String createKey(Broker broker, int timeout, int bufferSize, String clientId) {
-        return broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).connectionString() + "_" + timeout + "_" + bufferSize + "_" + clientId;
+        return broker.getConnectionString() + "_" + timeout + "_" + bufferSize + "_" + clientId;
     }
 
     public static TopicMeta getKafkaTopicMeta(KafkaClusterConfig kafkaClusterConfig) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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 ee5bb20..24eaa05 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,7 +22,6 @@ 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;
@@ -56,7 +55,7 @@ public final class KafkaUtils {
             if (partitionMetadata.errorCode() != 0) {
                 logger.warn("PartitionMetadata errorCode: " + partitionMetadata.errorCode());
             }
-            return new Broker(partitionMetadata.leader(), SecurityProtocol.PLAINTEXT);
+            return partitionMetadata.leader();
         } else {
             return null;
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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 f285153..c7de287 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.metadata.model.ISegment;
+import org.apache.kylin.cube.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/506cd783/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 da087c9..c318cba 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.metadata.model.ISegment;
+import org.apache.kylin.cube.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/506cd783/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 5692000..f1e5dab 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.metadata.model.ISegment;
+import org.apache.kylin.cube.ISegment;
 import org.apache.kylin.cube.cuboid.Cuboid;
 import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.dimension.DimensionEncoding;


[06/13] kylin git commit: Revert "refactor BuildCubeWithStream"

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

This reverts commit a08dd2e03900b321617647d1dbf1c4ee8b4b18c2.


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

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

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


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

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

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


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

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

This reverts commit 42dafc15db40731582d6257c618eff29643930a8.


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

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

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


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

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

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

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

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


[02/13] kylin git commit: Revert "KYLIN-1726 Scalable streaming cubing"

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

This reverts commit 81c7323b633df88eedac8b319fc57f9b62b01a4a.


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

Branch: refs/heads/master
Commit: 506cd783132023a06f1669ad248b74bf9d96d0e1
Parents: 1f48804
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:55:54 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 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 +-
 .../kylin/source/kafka/KafkaConfigManager.java  |  46 ++--
 .../apache/kylin/source/kafka/KafkaMRInput.java | 221 -------------------
 .../apache/kylin/source/kafka/KafkaSource.java  |  57 -----
 .../kylin/source/kafka/KafkaStreamingInput.java |  65 +++---
 .../kylin/source/kafka/MergeOffsetStep.java     |  89 --------
 .../kylin/source/kafka/SeekOffsetStep.java      | 119 ----------
 .../kylin/source/kafka/StreamingParser.java     |  49 ++--
 .../source/kafka/StringStreamingParser.java     |  49 ++--
 .../source/kafka/TimedJsonStreamParser.java     |  49 ++--
 .../apache/kylin/source/kafka/TopicMeta.java    |  49 ++--
 .../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 |  56 +++--
 .../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 +-
 45 files changed, 348 insertions(+), 2135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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..8c64f91 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -143,12 +143,14 @@ public class DeployUtil {
         deployHiveTables();
     }
 
-    public static void prepareTestDataForStreamingCube(long startTime, long endTime, int numberOfRecords, String cubeName, StreamDataLoader streamDataLoader) throws IOException {
+    public static void prepareTestDataForStreamingCube(long startTime, long endTime, String cubeName, StreamDataLoader streamDataLoader) throws IOException {
         CubeInstance cubeInstance = CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
-        List<String> data = StreamingTableDataGenerator.generate(numberOfRecords, startTime, endTime, cubeInstance.getFactTable());
+        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
         streamDataLoader.loadIntoKafka(data);
+        streamDataLoader.loadIntoKafka(data2);
         logger.info("Write {} messages into {}", data.size(), streamDataLoader.toString());
 
         //csv data for H2 use
@@ -163,7 +165,7 @@ public class DeployUtil {
             sb.append(StringUtils.join(rowColumns, ","));
             sb.append(System.getProperty("line.separator"));
         }
-        appendFactTableData(sb.toString(), cubeInstance.getFactTable());
+        overrideFactTableData(sb.toString(), cubeInstance.getFactTable());
     }
 
     public static void overrideFactTableData(String factTableContent, String factTableName) throws IOException {
@@ -177,33 +179,6 @@ 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/506cd783/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
deleted file mode 100644
index a5132af..0000000
--- a/assembly/src/test/java/org/apache/kylin/job/streaming/Kafka10DataLoader.java
+++ /dev/null
@@ -1,80 +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.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/506cd783/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 fafb1fc..79ee084 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
@@ -717,7 +717,6 @@ 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/506cd783/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..79397c3 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,7 +37,6 @@ 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/506cd783/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
new file mode 100644
index 0000000..2e1f214
--- /dev/null
+++ b/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java
@@ -0,0 +1,39 @@
+/*
+ * 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/506cd783/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 889a0b2..e31111d 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.metadata.model.ISegment;
+import org.apache.kylin.cube.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/506cd783/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..6aeb617 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,7 +26,6 @@ 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;
 
@@ -163,9 +162,4 @@ public class CubeJoinedFlatTableDesc implements IJoinedFlatTableDesc {
         return cubeDesc.getDistributedByColumn();
     }
 
-    @Override
-    public ISegment getSegment() {
-        return cubeSegment;
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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 8af2297..5212859 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,7 +25,6 @@ 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;
 
@@ -138,9 +137,4 @@ 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/506cd783/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 4213cf3..bb7503a 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
@@ -24,7 +24,7 @@ import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.util.Iterator;
 
-import org.apache.kylin.metadata.model.ISegment;
+import org.apache.kylin.cube.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/506cd783/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 ffa2680..f3a4107 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,6 +37,4 @@ public interface IJoinedFlatTableDesc {
     long getSourceOffsetEnd();
     
     TblColRef getDistributedBy();
-
-    ISegment getSegment();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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
deleted file mode 100644
index f69ae3f..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ISegment.java
+++ /dev/null
@@ -1,36 +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.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/506cd783/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 badf628..129d525 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,12 +34,10 @@ 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() {
@@ -57,7 +55,6 @@ 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/506cd783/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 62cede9..582052f 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,7 +21,6 @@ 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;
 
 /**
@@ -35,9 +34,6 @@ 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.
      */
@@ -71,10 +67,4 @@ 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/506cd783/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 67eef5e..2c3b77f 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,10 +71,6 @@ 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/506cd783/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..cfb889a 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/506cd783/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..9490560 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,36 +20,24 @@ 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.cube.CubeSegment;
-import org.apache.kylin.cube.CubeUpdate;
-import org.apache.kylin.engine.EngineFactory;
+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.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.job.streaming.KafkaDataLoader;
+import org.apache.kylin.metadata.realization.RealizationType;
 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.apache.kylin.storage.hbase.util.StorageCleanupJob;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -58,123 +46,31 @@ import org.slf4j.LoggerFactory;
  */
 public class BuildCubeWithStream {
 
-    private static final Logger logger = LoggerFactory.getLogger(org.apache.kylin.provision.BuildCubeWithStream.class);
-
-    private CubeManager cubeManager;
-    private DefaultScheduler scheduler;
-    protected ExecutableManager jobService;
+    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 KafkaConfig kafkaConfig;
-    private MockKafka kafkaServer;
-
-    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 {
-        Kafka10DataLoader dataLoader = new Kafka10DataLoader(kafkaConfig);
-        DeployUtil.prepareTestDataForStreamingCube(startTime, endTime, numberOfRecords, cubeName, dataLoader);
-        logger.info("Test data inserted into Kafka");
-    }
+    private KylinConfig kylinConfig;
 
-    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();
-    }
+    public static void main(String[] args) throws Exception {
 
-    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();
-    }
+        try {
+            beforeClass();
 
-    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();
-    }
+            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);
+        }
 
-    protected void deployEnv() throws IOException {
-        DeployUtil.overrideJobJarLocations();
-        //DeployUtil.initCliWorkDir();
-        //DeployUtil.deployMetadata();
     }
 
     public static void beforeClass() throws Exception {
@@ -187,54 +83,44 @@ public class BuildCubeWithStream {
         HBaseMetadataTestCase.staticCreateTestMetadata(HBaseMetadataTestCase.SANDBOX_TEST_DATA);
     }
 
-    public static void afterClass() throws Exception {
-        HBaseMetadataTestCase.staticCleanupTestMetadata();
+    protected void deployEnv() throws IOException {
+        DeployUtil.overrideJobJarLocations();
     }
 
-    public void after() {
-        kafkaServer.stop();
-    }
+    public void before() throws Exception {
+        deployEnv();
 
-    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();
-                }
-            }
-        }
-    }
+        kylinConfig = KylinConfig.getInstanceFromEnv();
+        final CubeInstance cubeInstance = CubeManager.getInstance(kylinConfig).getCube(cubeName);
+        final String factTable = cubeInstance.getFactTable();
+        final StreamingConfig config = StreamingManager.getInstance(kylinConfig).getStreamingConfig(factTable);
 
-    public static void main(String[] args) throws Exception {
-        try {
-            beforeClass();
+        //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);
 
-            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);
-        }
+        DeployUtil.prepareTestDataForStreamingCube(startTime, endTime, cubeName, new KafkaDataLoader(streamingConfig));
+    }
 
+    public void cleanup() throws Exception {
+        cleanupOldStorage();
+        HBaseMetadataTestCase.staticCleanupTestMetadata();
     }
 
     protected int cleanupOldStorage() throws Exception {
         String[] args = { "--delete", "true" };
 
-        //        KapStorageCleanupCLI cli = new KapStorageCleanupCLI();
-        //        cli.execute(args);
+        StorageCleanupJob cli = new StorageCleanupJob();
+        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/506cd783/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
deleted file mode 100644
index 3f47923..0000000
--- a/kylin-it/src/test/java/org/apache/kylin/provision/MockKafka.java
+++ /dev/null
@@ -1,191 +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.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/506cd783/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
deleted file mode 100644
index 98f6d04..0000000
--- a/kylin-it/src/test/java/org/apache/kylin/provision/NetworkUtils.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-package org.apache.kylin.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/506cd783/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 30d3324..1abc4eb 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.10.0.0</kafka.version>
+        <kafka.version>0.8.1</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/506cd783/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 09ac522..520d7cc 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,7 +50,6 @@ 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;
@@ -70,16 +69,6 @@ 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/506cd783/source-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/source-kafka/pom.xml b/source-kafka/pom.xml
index 212f4c6..90c2211 100644
--- a/source-kafka/pom.xml
+++ b/source-kafka/pom.xml
@@ -32,11 +32,10 @@
 
     </parent>
 
+    <properties>
+    </properties>
+
     <dependencies>
-        <dependency>
-            <groupId>org.apache.kylin</groupId>
-            <artifactId>kylin-engine-mr</artifactId>
-        </dependency>
 
         <dependency>
             <groupId>org.apache.kylin</groupId>
@@ -61,10 +60,16 @@
             <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/506cd783/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 cfdf316..d594873 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,19 +1,35 @@
 /*
- * 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/506cd783/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
deleted file mode 100644
index cfce137..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
+++ /dev/null
@@ -1,221 +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.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/506cd783/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
deleted file mode 100644
index d039583..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaSource.java
+++ /dev/null
@@ -1,57 +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.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/506cd783/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 de42689..c3bdb75 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,20 +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.
-*/
+ *
+ *
+ *  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;
@@ -24,9 +40,6 @@ 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;
@@ -53,8 +66,6 @@ import kafka.javaapi.FetchResponse;
 import kafka.javaapi.PartitionMetadata;
 import kafka.message.MessageAndOffset;
 
-import javax.annotation.Nullable;
-
 @SuppressWarnings("unused")
 public class KafkaStreamingInput implements IStreamingInput {
 
@@ -140,16 +151,8 @@ public class KafkaStreamingInput implements IStreamingInput {
                 if (partitionMetadata.errorCode() != 0) {
                     logger.warn("PartitionMetadata errorCode: " + partitionMetadata.errorCode());
                 }
-                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);
+                replicaBrokers = partitionMetadata.replicas();
+                return partitionMetadata.leader();
             } else {
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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
deleted file mode 100644
index a21b980..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/MergeOffsetStep.java
+++ /dev/null
@@ -1,89 +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.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/506cd783/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
deleted file mode 100644
index 5dca93f..0000000
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
+++ /dev/null
@@ -1,119 +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.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/506cd783/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 6b7d658..cb6a72b 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,20 +1,37 @@
 /*
- * 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.lang.reflect.Constructor;

http://git-wip-us.apache.org/repos/asf/kylin/blob/506cd783/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 666297f..8888d67 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,20 +1,37 @@
 /*
- * 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.nio.ByteBuffer;


[05/13] kylin git commit: KYLIN-2031 some more DimensionEncoding

Posted by ma...@apache.org.
KYLIN-2031 some more DimensionEncoding


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

Branch: refs/heads/master
Commit: a6cd40940248a99803d1596a32b62767646e39ac
Parents: 20ac92a
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 19:18:06 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/util/BytesUtil.java | 110 +++++-----
 .../gridtable/DimEncodingPreserveOrderTest.java | 113 ++++++++++
 .../dimension/DimensionEncodingFactory.java     |   2 +
 .../kylin/dimension/FixedLenHexDimEnc.java      | 149 ++++++++++---
 .../apache/kylin/dimension/IntegerDimEnc.java   |   3 +
 .../kylin/dimension/OneMoreByteVLongDimEnc.java | 206 +++++++++++++++++
 .../apache/kylin/dimension/SlimLongDimEnc.java  | 219 +++++++++++++++++++
 .../org/apache/kylin/dimension/VLongDimEnc.java | 196 -----------------
 .../kylin/dimension/FixedLenHexDimEncTest.java  | 182 +++++++++++++++
 .../kylin/dimension/IntegerDimEncTest.java      |   3 +
 .../dimension/OneMoreByteVLongDimEncTest.java   | 135 ++++++++++++
 .../kylin/dimension/SlimLongDimEncTest.java     | 151 +++++++++++++
 12 files changed, 1191 insertions(+), 278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
index 759ddbd..4ca6e64 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/BytesUtil.java
@@ -25,6 +25,13 @@ public class BytesUtil {
 
     public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
 
+    public static void writeByte(byte num, byte[] bytes, int offset, int size) {
+        for (int i = offset + size - 1; i >= offset; i--) {
+            bytes[i] = num;
+            num >>>= 8;
+        }
+    }
+
     public static void writeShort(short num, byte[] bytes, int offset, int size) {
         for (int i = offset + size - 1; i >= offset; i--) {
             bytes[i] = (byte) num;
@@ -41,56 +48,85 @@ public class BytesUtil {
         return num;
     }
 
-    public static void writeLong(long num, byte[] bytes, int offset, int size) {
+    public static void writeUnsigned(int num, byte[] bytes, int offset, int size) {
         for (int i = offset + size - 1; i >= offset; i--) {
             bytes[i] = (byte) num;
             num >>>= 8;
         }
     }
 
-    public static long readLong(byte[] bytes, int offset, int size) {
-        long integer = 0;
+    public static int readUnsigned(byte[] bytes, int offset, int size) {
+        int integer = 0;
         for (int i = offset, n = offset + size; i < n; i++) {
             integer <<= 8;
-            integer |= (long) bytes[i] & 0xFF;
+            integer |= (int) bytes[i] & 0xFF;
         }
         return integer;
     }
 
-    public static long readLong(ByteBuffer buffer, int size) {
-        long integer = 0;
+    public static void writeUnsigned(int num, int size, ByteBuffer out) {
+        int mask = 0xff << ((size - 1) * 8);
+        for (int i = size; i > 0; i--) {
+            int v = (num & mask) >> (i - 1) * 8;
+            out.put((byte) v);
+            mask = mask >> 8;
+        }
+    }
+
+    public static int readUnsigned(ByteBuffer in, int size) {
+        int integer = 0;
         for (int i = 0; i < size; i++) {
+            integer = integer << 8;
+            integer |= (in.get() & 0xff);
+        }
+
+        return integer;
+    }
+
+    public static void writeSignedLong(long num, byte[] bytes, int offset, int size) {
+        writeLong(num, bytes, offset, size);
+    }
+
+    public static long readSignedLong(byte[] bytes, int offset, int size) {
+        long integer = (bytes[offset] & 0x80) == 0 ? 0 : -1;
+        for (int i = offset, n = offset + size; i < n; i++) {
             integer <<= 8;
-            integer |= (long) buffer.get() & 0xFF;
+            integer |= (int) bytes[i] & 0xFF;
         }
         return integer;
     }
 
-    public static void writeUnsigned(int num, byte[] bytes, int offset, int size) {
+    public static void writeLong(long num, byte[] bytes, int offset, int size) {
         for (int i = offset + size - 1; i >= offset; i--) {
             bytes[i] = (byte) num;
             num >>>= 8;
         }
     }
 
-    public static int readUnsigned(byte[] bytes, int offset, int size) {
-        int integer = 0;
+    public static long readLong(byte[] bytes, int offset, int size) {
+        long integer = 0;
         for (int i = offset, n = offset + size; i < n; i++) {
             integer <<= 8;
-            integer |= (int) bytes[i] & 0xFF;
+            integer |= (long) bytes[i] & 0xFF;
         }
         return integer;
     }
 
-    public static void writeSigned(int num, byte[] bytes, int offset, int size) {
-        writeUnsigned(num, bytes, offset, size);
+    public static void writeLong(long num, ByteBuffer out) {
+        for (int i = 0; i < 8; i++) {
+            out.put((byte) num);
+            num >>>= 8;
+        }
     }
 
-    public static int readSigned(byte[] bytes, int offset, int size) {
-        int integer = (bytes[offset] & 0x80) == 0 ? 0 : -1;
-        for (int i = offset, n = offset + size; i < n; i++) {
-            integer <<= 8;
-            integer |= (int) bytes[i] & 0xFF;
+    public static long readLong(ByteBuffer in) {
+        long integer = 0;
+        int mask = 0xff;
+        int shift = 0;
+        for (int i = 0; i < 8; i++) {
+            integer |= (in.get() << shift) & mask;
+            mask = mask << 8;
+            shift += 8;
         }
         return integer;
     }
@@ -204,44 +240,6 @@ public class BytesUtil {
         return -111 - value;
     }
 
-    public static void writeUnsigned(int num, int size, ByteBuffer out) {
-        int mask = 0xff << ((size - 1) * 8);
-        for (int i = size; i > 0; i--) {
-            int v = (num & mask) >> (i - 1) * 8;
-            out.put((byte) v);
-            mask = mask >> 8;
-        }
-    }
-
-    public static int readUnsigned(ByteBuffer in, int size) {
-        int integer = 0;
-        for (int i = 0; i < size; i++) {
-            integer = integer << 8;
-            integer |= (in.get() & 0xff);
-        }
-
-        return integer;
-    }
-
-    public static void writeLong(long num, ByteBuffer out) {
-        for (int i = 0; i < 8; i++) {
-            out.put((byte) num);
-            num >>>= 8;
-        }
-    }
-
-    public static long readLong(ByteBuffer in) {
-        long integer = 0;
-        int mask = 0xff;
-        int shift = 0;
-        for (int i = 0; i < 8; i++) {
-            integer |= (in.get() << shift) & mask;
-            mask = mask << 8;
-            shift += 8;
-        }
-        return integer;
-    }
-
     public static void writeUTFString(String str, ByteBuffer out) {
         byte[] bytes = str == null ? null : Bytes.toBytes(str);
         writeByteArray(bytes, out);

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
new file mode 100644
index 0000000..0c84ed0
--- /dev/null
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/DimEncodingPreserveOrderTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.gridtable;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.dimension.DimensionEncoding;
+import org.apache.kylin.dimension.FixedLenHexDimEnc;
+import org.apache.kylin.dimension.OneMoreByteVLongDimEnc;
+import org.apache.kylin.dimension.SlimLongDimEnc;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Ordering;
+
+public class DimEncodingPreserveOrderTest {
+    @Test
+    public void testOneMoreByteVLongDimEncPreserveOrder() {
+        OneMoreByteVLongDimEnc enc = new OneMoreByteVLongDimEnc(2);
+        List<ByteArray> encodedValues = Lists.newArrayList();
+        encodedValues.add(encode(enc, -32768L));
+        encodedValues.add(encode(enc, -10000L));
+        encodedValues.add(encode(enc, -100L));
+        encodedValues.add(encode(enc, 0L));
+        encodedValues.add(encode(enc, 100L));
+        encodedValues.add(encode(enc, 10000L));
+        encodedValues.add(encode(enc, 32767L));
+        encodedValues.add(encode(enc, null));
+
+        assertTrue(Ordering.from(new DefaultGTComparator()).isOrdered(encodedValues));
+    }
+
+    @Test
+    public void testVLongDimEncPreserveOrder() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(2);
+        List<ByteArray> encodedValues = Lists.newArrayList();
+        encodedValues.add(encode(enc, -32767L));
+        encodedValues.add(encode(enc, -10000L));
+        encodedValues.add(encode(enc, -100L));
+        encodedValues.add(encode(enc, 0L));
+        encodedValues.add(encode(enc, 100L));
+        encodedValues.add(encode(enc, 10000L));
+        encodedValues.add(encode(enc, 32767L));
+        encodedValues.add(encode(enc, null));
+
+        assertTrue(Ordering.from(new DefaultGTComparator()).isOrdered(encodedValues));
+    }
+
+    @Test
+    public void testVLongDimEncPreserveOrder2() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(8);
+        List<ByteArray> encodedValues = Lists.newArrayList();
+        encodedValues.add(encode(enc, -Long.MAX_VALUE));
+        encodedValues.add(encode(enc, -10000L));
+        encodedValues.add(encode(enc, -100L));
+        encodedValues.add(encode(enc, 0L));
+        encodedValues.add(encode(enc, 100L));
+        encodedValues.add(encode(enc, 10000L));
+        encodedValues.add(encode(enc, Long.MAX_VALUE));
+        encodedValues.add(encode(enc, null));
+
+        assertTrue(Ordering.from(new DefaultGTComparator()).isOrdered(encodedValues));
+    }
+
+    private ByteArray encode(DimensionEncoding enc, Object value) {
+        if (value != null) {
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+
+            String valueStr = "" + value;
+            byte[] bytes = Bytes.toBytes(valueStr);
+
+            enc.encode(bytes, bytes.length, buf, 0);
+            return new ByteArray(buf);
+        } else {
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            return new ByteArray(buf);
+        }
+    }
+
+    @Test
+    public void testFixedLengthHexDimEncPreserveOrder() {
+        FixedLenHexDimEnc enc = new FixedLenHexDimEnc(4);
+        List<ByteArray> encodedValues = Lists.newArrayList();
+        encodedValues.add(encode(enc, "0000"));
+        encodedValues.add(encode(enc, "0001"));
+        encodedValues.add(encode(enc, "FFF0"));
+        encodedValues.add(encode(enc, null));
+
+        assertTrue(Ordering.from(new DefaultGTComparator()).isOrdered(encodedValues));
+    }
+
+}

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

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
index 71fe376..fbf1ccf 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
@@ -29,18 +29,75 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * used to store hex values like "1A2BFF"
+ * <p>
+ * <p>
+ * limitations: (take FixedLenHexDimEnc(2) as example: )
+ * <p>
+ * 1. "FFFF" will become null encode and decode
+ * 2. "AB" will become "AB00"
  * 
- * FixedLenHexDimEnc does not support expressing NULL values, nulls will become "FFFFFF" after encode and decode
+ * <p>
+ * Due to these limitations hex representation of hash values(with no padding, better with even characters) is more suitable
  */
 public class FixedLenHexDimEnc extends DimensionEncoding {
     private static final long serialVersionUID = 1L;
 
     private static Logger logger = LoggerFactory.getLogger(FixedLenHexDimEnc.class);
 
+    public static byte[] dict = new byte[256];
+    public static byte[] revdict = new byte[16];
+
+    static {
+        for (int i = 0; i < dict.length; i++) {
+            dict[i] = -1;
+        }
+        dict['0'] = 0;
+        dict['1'] = 1;
+        dict['2'] = 2;
+        dict['3'] = 3;
+        dict['4'] = 4;
+        dict['5'] = 5;
+        dict['6'] = 6;
+        dict['7'] = 7;
+        dict['8'] = 8;
+        dict['9'] = 9;
+        dict['A'] = 10;
+        dict['B'] = 11;
+        dict['C'] = 12;
+        dict['D'] = 13;
+        dict['E'] = 14;
+        dict['F'] = 15;
+        dict['a'] = 10;
+        dict['b'] = 11;
+        dict['c'] = 12;
+        dict['d'] = 13;
+        dict['e'] = 14;
+        dict['f'] = 15;
+
+        revdict[0] = '0';
+        revdict[1] = '1';
+        revdict[2] = '2';
+        revdict[3] = '3';
+        revdict[4] = '4';
+        revdict[5] = '5';
+        revdict[6] = '6';
+        revdict[7] = '7';
+        revdict[8] = '8';
+        revdict[9] = '9';
+        revdict[10] = 'A';
+        revdict[11] = 'B';
+        revdict[12] = 'C';
+        revdict[13] = 'D';
+        revdict[14] = 'E';
+        revdict[15] = 'F';
+    }
+
     // row key fixed length place holder
-    public static final byte ROWKEY_PLACE_HOLDER_BYTE = 9;
+    public static final byte ROWKEY_PLACE_HOLDER_BYTE = 0;
 
     public static final String ENCODING_NAME = "fixed_length_hex";
 
@@ -54,22 +111,26 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
         public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
             return new FixedLenHexDimEnc(Integer.parseInt(args[0]));
         }
-    };
+    }
 
     // ============================================================================
 
-    private int fixedLen;
+    private int hexLength;
     private int bytelen;
 
     transient private int avoidVerbose = 0;
+    transient private int avoidVerbose2 = 0;
 
     //no-arg constructor is required for Externalizable
     public FixedLenHexDimEnc() {
     }
 
     public FixedLenHexDimEnc(int len) {
-        this.fixedLen = len;
-        this.bytelen = (fixedLen + 1) / 2;
+        if (len < 1) {
+            throw new IllegalArgumentException("len has to be positive: " + len);
+        }
+        this.hexLength = len;
+        this.bytelen = (hexLength + 1) / 2;
     }
 
     @Override
@@ -81,51 +142,78 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
 
         FixedLenHexDimEnc that = (FixedLenHexDimEnc) o;
 
-        return fixedLen == that.fixedLen;
-
+        return hexLength == that.hexLength;
     }
 
     @Override
     public int hashCode() {
-        return fixedLen;
+        return hexLength;
     }
 
     @Override
     public int getLengthOfEncoding() {
-        return fixedLen;
+        return bytelen;
     }
 
     @Override
     public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
         if (value == null) {
-            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            Arrays.fill(output, outputOffset, outputOffset + bytelen, NULL);
             return;
         }
 
-        if (valueLen > fixedLen) {
+        int endOffset = outputOffset + bytelen;
+
+        if (valueLen > hexLength) {
             if (avoidVerbose++ % 10000 == 0) {
-                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueLen + ", will truncate, value string: " + Bytes.toString(value, 0, valueLen) + " times:" + avoidVerbose);
+                logger.warn("Expect at most " + hexLength + " bytes, but got " + valueLen + ", will truncate, value string: " + Bytes.toString(value, 0, valueLen) + " times:" + avoidVerbose);
+            }
+        }
+
+        if (valueLen >= hexLength && isF(value, 0, hexLength)) {
+            if (avoidVerbose2++ % 10000 == 0) {
+                logger.warn("All 'F' value: " + Bytes.toString(value, 0, valueLen) + "will become null after encode/decode. times:" + avoidVerbose);
             }
         }
 
-        int n = Math.min(valueLen, fixedLen);
-        System.arraycopy(value, 0, output, outputOffset, n);
+        int n = Math.min(valueLen, hexLength);
+        for (int i = 0; i < n; i += 2) {
+            byte temp = 0;
+            byte iCode = dict[value[i]];
+            temp |= (iCode << 4);
 
-        if (n < fixedLen) {
-            Arrays.fill(output, outputOffset + n, outputOffset + fixedLen, ROWKEY_PLACE_HOLDER_BYTE);
+            int j = i + 1;
+            if (j < n) {
+                byte jCode = dict[value[j]];
+                temp |= jCode;
+            }
+
+            output[outputOffset++] = temp;
         }
+
+        Arrays.fill(output, outputOffset, endOffset, ROWKEY_PLACE_HOLDER_BYTE);
     }
 
     @Override
     public String decode(byte[] bytes, int offset, int len) {
+        Preconditions.checkArgument(len == bytelen, "len " + len + " not equals " + bytelen);
+
         if (isNull(bytes, offset, len)) {
             return null;
         }
 
-        while (len > 0 && bytes[offset + len - 1] == ROWKEY_PLACE_HOLDER_BYTE)
-            len--;
+        byte[] ret = new byte[hexLength];
+        for (int i = 0; i < ret.length; i += 2) {
+            byte temp = bytes[i / 2];
+            ret[i] = revdict[(temp & 0xF0) >>> 4];
 
-        return Bytes.toString(bytes, offset, len);
+            int j = i + 1;
+            if (j < hexLength) {
+                ret[j] = revdict[temp & 0x0F];
+            }
+        }
+
+        return Bytes.toString(ret, 0, ret.length);
     }
 
     @Override
@@ -140,7 +228,7 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
         private byte[] currentBuf() {
             byte[] buf = current.get();
             if (buf == null) {
-                buf = new byte[fixedLen];
+                buf = new byte[bytelen];
                 current.set(buf);
             }
             return buf;
@@ -163,17 +251,17 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
 
         @Override
         public int peekLength(ByteBuffer in) {
-            return fixedLen;
+            return bytelen;
         }
 
         @Override
         public int maxLength() {
-            return fixedLen;
+            return bytelen;
         }
 
         @Override
         public int getStorageBytesEstimate() {
-            return fixedLen;
+            return bytelen;
         }
 
         @Override
@@ -184,12 +272,21 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
 
     @Override
     public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeShort(fixedLen);
+        out.writeShort(hexLength);
     }
 
     @Override
     public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        fixedLen = in.readShort();
+        hexLength = in.readShort();
+    }
+
+    private boolean isF(byte[] value, int offset, int length) {
+        for (int i = offset; i < length + offset; ++i) {
+            if (value[i] != 'F') {
+                return false;
+            }
+        }
+        return true;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
index 89b14fc..b87e46a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
@@ -30,6 +30,9 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * deprecated use SlimLongDimEnc instead
+ */
 public class IntegerDimEnc extends DimensionEncoding {
     private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
new file mode 100644
index 0000000..3ba9d2f
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *  
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.dimension;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * not being used yet, prepared for future
+ */
+public class OneMoreByteVLongDimEnc extends DimensionEncoding {
+    private static final long serialVersionUID = 1L;
+
+    private static Logger logger = LoggerFactory.getLogger(OneMoreByteVLongDimEnc.class);
+
+    private static final long[] CAP = { 0, 0x7fL, 0x7fffL, 0x7fffffL, 0x7fffffffL, 0x7fffffffffL, 0x7fffffffffffL, 0x7fffffffffffffL, 0x7fffffffffffffffL };
+    private static final long[] MASK = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, 0xffffffffffffffffL };
+    private static final long[] TAIL = { 0, 0x80L, 0x8000L, 0x800000L, 0x80000000L, 0x8000000000L, 0x800000000000L, 0x80000000000000L, 0x8000000000000000L };
+    static {
+        for (int i = 1; i < TAIL.length; ++i) {
+            long head = ~MASK[i];
+            TAIL[i] = head | TAIL[i];
+        }
+    }
+    public static final String ENCODING_NAME = "one_more_byte_vlong";
+
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new OneMoreByteVLongDimEnc(Integer.parseInt(args[0]));
+        }
+    };
+
+    // ============================================================================
+
+    private int fixedLen;
+    private int byteLen;
+
+    transient private int avoidVerbose = 0;
+
+    //no-arg constructor is required for Externalizable
+    public OneMoreByteVLongDimEnc() {
+    }
+
+    public OneMoreByteVLongDimEnc(int len) {
+        if (len <= 0 || len >= CAP.length)
+            throw new IllegalArgumentException();
+
+        this.fixedLen = len;
+        this.byteLen = fixedLen + 1;//one additional byte to indicate null
+    }
+
+    @Override
+    public int getLengthOfEncoding() {
+        return byteLen;
+    }
+
+    @Override
+    public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
+        if (value == null) {
+            Arrays.fill(output, outputOffset, outputOffset + byteLen, NULL);
+            return;
+        }
+
+        encode(Bytes.toString(value, 0, valueLen), output, outputOffset);
+    }
+
+    void encode(String valueStr, byte[] output, int outputOffset) {
+        if (valueStr == null) {
+            Arrays.fill(output, outputOffset, outputOffset + byteLen, NULL);
+            return;
+        }
+
+        long integer = Long.parseLong(valueStr);
+        if (integer > CAP[fixedLen] || integer < TAIL[fixedLen]) {
+            if (avoidVerbose++ % 10000 == 0) {
+                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
+            }
+        }
+
+        BytesUtil.writeByte(integer >= 0 ? (byte) 1 : (byte) 0, output, outputOffset, 1);
+        BytesUtil.writeSignedLong(integer, output, outputOffset + 1, fixedLen);
+    }
+
+    @Override
+    public String decode(byte[] bytes, int offset, int len) {
+        if (isNull(bytes, offset, len)) {
+            return null;
+        }
+
+        long integer = BytesUtil.readSignedLong(bytes, offset + 1, len - 1);
+        return String.valueOf(integer);
+    }
+
+    @Override
+    public DataTypeSerializer<Object> asDataTypeSerializer() {
+        return new VLongSerializer();
+    }
+
+    public class VLongSerializer extends DataTypeSerializer<Object> {
+        // be thread-safe and avoid repeated obj creation
+        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
+
+        private byte[] currentBuf() {
+            byte[] buf = current.get();
+            if (buf == null) {
+                buf = new byte[byteLen];
+                current.set(buf);
+            }
+            return buf;
+        }
+
+        @Override
+        public void serialize(Object value, ByteBuffer out) {
+            byte[] buf = currentBuf();
+            String valueStr = value == null ? null : value.toString();
+            encode(valueStr, buf, 0);
+            out.put(buf);
+        }
+
+        @Override
+        public Object deserialize(ByteBuffer in) {
+            byte[] buf = currentBuf();
+            in.get(buf);
+            return decode(buf, 0, buf.length);
+        }
+
+        @Override
+        public int peekLength(ByteBuffer in) {
+            return byteLen;
+        }
+
+        @Override
+        public int maxLength() {
+            return byteLen;
+        }
+
+        @Override
+        public int getStorageBytesEstimate() {
+            return byteLen;
+        }
+
+        @Override
+        public Object valueOf(String str) {
+            return str;
+        }
+    }
+
+    @Override
+    public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeShort(fixedLen);
+    }
+
+    @Override
+    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        fixedLen = in.readShort();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        OneMoreByteVLongDimEnc that = (OneMoreByteVLongDimEnc) o;
+
+        return fixedLen == that.fixedLen;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return fixedLen;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
new file mode 100644
index 0000000..4ac871f
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/SlimLongDimEnc.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *  
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.dimension;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * replacement for IntegerDimEnc, the diff is VLongDimEnc supports negative values
+ */
+public class SlimLongDimEnc extends DimensionEncoding {
+    private static final long serialVersionUID = 1L;
+
+    private static Logger logger = LoggerFactory.getLogger(SlimLongDimEnc.class);
+
+    private static final long[] CAP = { 0, 0x7fL, 0x7fffL, 0x7fffffL, 0x7fffffffL, 0x7fffffffffL, 0x7fffffffffffL, 0x7fffffffffffffL, 0x7fffffffffffffffL };
+    private static final long[] MASK = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, 0xffffffffffffffffL };
+    private static final long[] TAIL = { 0, 0x80L, 0x8000L, 0x800000L, 0x80000000L, 0x8000000000L, 0x800000000000L, 0x80000000000000L, 0x8000000000000000L };
+    static {
+        for (int i = 1; i < TAIL.length; ++i) {
+            long head = ~MASK[i];
+            TAIL[i] = head | TAIL[i];
+        }
+    }
+
+    public static final String ENCODING_NAME = "slimlong";
+
+    public static class Factory extends DimensionEncodingFactory {
+        @Override
+        public String getSupportedEncodingName() {
+            return ENCODING_NAME;
+        }
+
+        @Override
+        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
+            return new SlimLongDimEnc(Integer.parseInt(args[0]));
+        }
+    };
+
+    // ============================================================================
+
+    private int fixedLen;
+
+    transient private int avoidVerbose = 0;
+    transient private int avoidVerbose2 = 0;
+
+    //no-arg constructor is required for Externalizable
+    public SlimLongDimEnc() {
+    }
+
+    public SlimLongDimEnc(int len) {
+        if (len <= 0 || len >= CAP.length)
+            throw new IllegalArgumentException();
+
+        this.fixedLen = len;
+    }
+
+    @Override
+    public int getLengthOfEncoding() {
+        return fixedLen;
+    }
+
+    @Override
+    public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
+        if (value == null) {
+            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            return;
+        }
+
+        encode(Bytes.toString(value, 0, valueLen), output, outputOffset);
+    }
+
+    void encode(String valueStr, byte[] output, int outputOffset) {
+        if (valueStr == null) {
+            Arrays.fill(output, outputOffset, outputOffset + fixedLen, NULL);
+            return;
+        }
+
+        long integer = Long.parseLong(valueStr);
+        if (integer > CAP[fixedLen] || integer < TAIL[fixedLen]) {
+            if (avoidVerbose++ % 10000 == 0) {
+                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
+            }
+        }
+
+        if (integer == TAIL[fixedLen]) {
+            if (avoidVerbose2++ % 10000 == 0) {
+                logger.warn("Value " + valueStr + " does not fit into " + fixedLen + " bytes ");
+            }
+        }
+
+        BytesUtil.writeLong(integer + CAP[fixedLen], output, outputOffset, fixedLen);//apply an offset to preserve binary order, overflow is okay
+    }
+
+    @Override
+    public String decode(byte[] bytes, int offset, int len) {
+        if (isNull(bytes, offset, len)) {
+            return null;
+        }
+
+        long integer = BytesUtil.readLong(bytes, offset, len) - CAP[fixedLen];
+
+        //only take useful bytes
+        integer = integer & MASK[fixedLen];
+        boolean positive = (integer & ((0x80) << ((fixedLen - 1) << 3))) == 0;
+        if (!positive) {
+            integer |= (~MASK[fixedLen]);
+        }
+
+        return String.valueOf(integer);
+    }
+
+    @Override
+    public DataTypeSerializer<Object> asDataTypeSerializer() {
+        return new IntegerSerializer();
+    }
+
+    public class IntegerSerializer extends DataTypeSerializer<Object> {
+        // be thread-safe and avoid repeated obj creation
+        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
+
+        private byte[] currentBuf() {
+            byte[] buf = current.get();
+            if (buf == null) {
+                buf = new byte[fixedLen];
+                current.set(buf);
+            }
+            return buf;
+        }
+
+        @Override
+        public void serialize(Object value, ByteBuffer out) {
+            byte[] buf = currentBuf();
+            String valueStr = value == null ? null : value.toString();
+            encode(valueStr, buf, 0);
+            out.put(buf);
+        }
+
+        @Override
+        public Object deserialize(ByteBuffer in) {
+            byte[] buf = currentBuf();
+            in.get(buf);
+            return decode(buf, 0, buf.length);
+        }
+
+        @Override
+        public int peekLength(ByteBuffer in) {
+            return fixedLen;
+        }
+
+        @Override
+        public int maxLength() {
+            return fixedLen;
+        }
+
+        @Override
+        public int getStorageBytesEstimate() {
+            return fixedLen;
+        }
+
+        @Override
+        public Object valueOf(String str) {
+            return str;
+        }
+    }
+
+    @Override
+    public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeShort(fixedLen);
+    }
+
+    @Override
+    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        fixedLen = in.readShort();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        SlimLongDimEnc that = (SlimLongDimEnc) o;
+
+        return fixedLen == that.fixedLen;
+
+    }
+
+    @Override
+    public int hashCode() {
+        return fixedLen;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/main/java/org/apache/kylin/dimension/VLongDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/VLongDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/VLongDimEnc.java
deleted file mode 100644
index 000cd92..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/VLongDimEnc.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *  
- *     http://www.apache.org/licenses/LICENSE-2.0
- *  
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.dimension;
-
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataTypeSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class VLongDimEnc extends DimensionEncoding {
-    private static final long serialVersionUID = 1L;
-
-    private static Logger logger = LoggerFactory.getLogger(VLongDimEnc.class);
-
-    private static final long[] CAP = { 0, 0xffL, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, Long.MAX_VALUE };
-    private static final long[] TAIL = { 0, 0x80L, 0xffffL, 0xffffffL, 0xffffffffL, 0xffffffffffL, 0xffffffffffffL, 0xffffffffffffffL, Long.MAX_VALUE };
-
-    public static final String ENCODING_NAME = "vlong";
-
-    public static class Factory extends DimensionEncodingFactory {
-        @Override
-        public String getSupportedEncodingName() {
-            return ENCODING_NAME;
-        }
-
-        @Override
-        public DimensionEncoding createDimensionEncoding(String encodingName, String[] args) {
-            return new VLongDimEnc(Integer.parseInt(args[0]));
-        }
-    };
-
-    // ============================================================================
-
-    private int fixedLen;
-    private int byteLen;
-
-    transient private int avoidVerbose = 0;
-
-    //no-arg constructor is required for Externalizable
-    public VLongDimEnc() {
-    }
-
-    public VLongDimEnc(int len) {
-        if (len <= 0 || len >= CAP.length)
-            throw new IllegalArgumentException();
-
-        this.fixedLen = len;
-        this.byteLen = (fixedLen + 1) / 2 + 1;//one additional byte to indicate null
-    }
-
-    @Override
-    public int getLengthOfEncoding() {
-        return byteLen;
-    }
-
-    @Override
-    public void encode(byte[] value, int valueLen, byte[] output, int outputOffset) {
-        if (value == null) {
-            Arrays.fill(output, outputOffset, outputOffset + byteLen, NULL);
-            return;
-        }
-
-        encode(Bytes.toString(value, 0, valueLen), output, outputOffset);
-    }
-
-    void encode(String valueStr, byte[] output, int outputOffset) {
-        if (valueStr == null) {
-            Arrays.fill(output, outputOffset, outputOffset + byteLen, NULL);
-            return;
-        }
-
-        long integer = Long.parseLong(valueStr);
-        if (integer > CAP[fixedLen]) {
-            if (avoidVerbose++ % 10000 == 0) {
-                logger.warn("Expect at most " + fixedLen + " bytes, but got " + valueStr + ", will truncate, hit times:" + avoidVerbose);
-            }
-        }
-
-        BytesUtil.writeLong(integer, output, outputOffset, fixedLen);
-    }
-
-    @Override
-    public String decode(byte[] bytes, int offset, int len) {
-        if (isNull(bytes, offset, len)) {
-            return null;
-        }
-
-        long integer = BytesUtil.readLong(bytes, offset, len);
-        return String.valueOf(integer);
-    }
-
-    @Override
-    public DataTypeSerializer<Object> asDataTypeSerializer() {
-        return new IntegerSerializer();
-    }
-
-    public class IntegerSerializer extends DataTypeSerializer<Object> {
-        // be thread-safe and avoid repeated obj creation
-        private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
-
-        private byte[] currentBuf() {
-            byte[] buf = current.get();
-            if (buf == null) {
-                buf = new byte[fixedLen];
-                current.set(buf);
-            }
-            return buf;
-        }
-
-        @Override
-        public void serialize(Object value, ByteBuffer out) {
-            byte[] buf = currentBuf();
-            String valueStr = value == null ? null : value.toString();
-            encode(valueStr, buf, 0);
-            out.put(buf);
-        }
-
-        @Override
-        public Object deserialize(ByteBuffer in) {
-            byte[] buf = currentBuf();
-            in.get(buf);
-            return decode(buf, 0, buf.length);
-        }
-
-        @Override
-        public int peekLength(ByteBuffer in) {
-            return fixedLen;
-        }
-
-        @Override
-        public int maxLength() {
-            return fixedLen;
-        }
-
-        @Override
-        public int getStorageBytesEstimate() {
-            return fixedLen;
-        }
-
-        @Override
-        public Object valueOf(String str) {
-            return str;
-        }
-    }
-
-    @Override
-    public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeShort(fixedLen);
-    }
-
-    @Override
-    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        fixedLen = in.readShort();
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        VLongDimEnc that = (VLongDimEnc) o;
-
-        return fixedLen == that.fixedLen;
-
-    }
-
-    @Override
-    public int hashCode() {
-        return fixedLen;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/test/java/org/apache/kylin/dimension/FixedLenHexDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/FixedLenHexDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/FixedLenHexDimEncTest.java
new file mode 100644
index 0000000..6f0540b
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/FixedLenHexDimEncTest.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *  
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.dimension;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class FixedLenHexDimEncTest {
+
+    @Test
+    public void testConstructor() {
+        try {
+            new FixedLenHexDimEnc(0);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+
+        new FixedLenHexDimEnc(8);
+    }
+
+    @Test
+    public void testNull() {
+        for (int i = 1; i < 9; i++) {
+            FixedLenHexDimEnc enc = new FixedLenHexDimEnc(i);
+
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            String decode = enc.decode(buf, 0, buf.length);
+            Assert.assertEquals(null, decode);
+
+            buf = new byte[enc.getLengthOfEncoding()];
+            DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+            ser.serialize(null, ByteBuffer.wrap(buf));
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+            Assert.assertEquals(null, decode);
+        }
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        FixedLenHexDimEnc enc = new FixedLenHexDimEnc(4);
+        testEncodeDecode(enc, "AF12");
+        testEncodeDecode(enc, "0000");
+        testEncodeDecode(enc, "FFF0");
+        try {
+            testEncodeDecode(enc, "abcd");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[abcd]> but was:<[ABCD]>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF> but was:<null>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF[]> but was:<FFF[0]>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFFF0");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF0> but was:<null>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFF10");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF1[0]> but was:<FFF1[]>", e.getMessage());
+        }
+    }
+
+
+    @Test
+    public void testEncodeDecode2() {
+        FixedLenHexDimEnc enc = new FixedLenHexDimEnc(5);
+        testEncodeDecode(enc, "AF121");
+        testEncodeDecode(enc, "00000");
+        
+        //with a little extra room all F is supported
+        testEncodeDecode(enc, "FFFFF");
+        
+        try {
+            testEncodeDecode(enc, "FFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF[]> but was:<FFF[00]>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFFFF0");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFFF[0]> but was:<FFFFF[]>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, "FFFF10");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF1[0]> but was:<FFFF1[]>", e.getMessage());
+        }
+
+    }
+
+    private void testEncodeDecode(FixedLenHexDimEnc enc, String value) {
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = value;
+        byte[] bytes = Bytes.toBytes(valueStr);
+        enc.encode(bytes, bytes.length, buf, 0);
+        String decode = enc.decode(buf, 0, buf.length);
+        Assert.assertEquals(valueStr, decode);
+    }
+
+    @Test
+    public void testSerDes() {
+
+        FixedLenHexDimEnc enc = new FixedLenHexDimEnc(4);
+        testSerDes(enc, "AF12");
+        testSerDes(enc, "0000");
+        testSerDes(enc, "FFF0");
+        try {
+            testSerDes(enc, "FFFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF> but was:<null>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, "FFF");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF[]> but was:<FFF[0]>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, "FFFF0");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFFF0> but was:<null>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, "FFF10");
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<FFF1[0]> but was:<FFF1[]>", e.getMessage());
+        }
+    }
+
+    private void testSerDes(FixedLenHexDimEnc enc, String value) {
+        DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = value;
+        ser.serialize(valueStr, ByteBuffer.wrap(buf));
+        String decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+        Assert.assertEquals(valueStr, decode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
index 34f0427..fccd8b9 100644
--- a/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/IntegerDimEncTest.java
@@ -25,6 +25,9 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
 import org.junit.Assert;
 import org.junit.Test;
 
+/**
+ * Deprecated. use VLongDimEnc instead
+ */
 public class IntegerDimEncTest {
 
     @Test

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/test/java/org/apache/kylin/dimension/OneMoreByteVLongDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/OneMoreByteVLongDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/OneMoreByteVLongDimEncTest.java
new file mode 100644
index 0000000..ba6ab55
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/OneMoreByteVLongDimEncTest.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *  
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.dimension;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class OneMoreByteVLongDimEncTest {
+
+    @Test
+    public void testConstructor() {
+        try {
+            new OneMoreByteVLongDimEnc(0);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        try {
+            new OneMoreByteVLongDimEnc(9);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        new OneMoreByteVLongDimEnc(8);
+    }
+
+    @Test
+    public void testNull() {
+        for (int i = 1; i < 9; i++) {
+            OneMoreByteVLongDimEnc enc = new OneMoreByteVLongDimEnc(i);
+
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            String decode = enc.decode(buf, 0, buf.length);
+            Assert.assertEquals(null, decode);
+
+            buf = new byte[enc.getLengthOfEncoding()];
+            DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+            ser.serialize(null, ByteBuffer.wrap(buf));
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+            Assert.assertEquals(null, decode);
+        }
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        OneMoreByteVLongDimEnc enc = new OneMoreByteVLongDimEnc(2);
+        testEncodeDecode(enc, 0);
+        testEncodeDecode(enc, 100);
+        testEncodeDecode(enc, 10000);
+        testEncodeDecode(enc, 32767);
+        testEncodeDecode(enc, -100);
+        testEncodeDecode(enc, -10000);
+        testEncodeDecode(enc, -32768);
+        try {
+            testEncodeDecode(enc, 32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[]32768> but was:<[-]32768>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, -32769);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[-32769]> but was:<[32767]>", e.getMessage());
+        }
+    }
+
+ 
+    private void testEncodeDecode(OneMoreByteVLongDimEnc enc, long value) {
+        String valueStr = "" + value;
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        byte[] bytes = Bytes.toBytes(valueStr);
+        enc.encode(bytes, bytes.length, buf, 0);
+        String decode = enc.decode(buf, 0, buf.length);
+        Assert.assertEquals(valueStr, decode);
+    }
+
+
+    @Test
+    public void testSerDes() {
+        OneMoreByteVLongDimEnc enc = new OneMoreByteVLongDimEnc(2);
+        testSerDes(enc, 0);
+        testSerDes(enc, 100);
+        testSerDes(enc, 10000);
+        testSerDes(enc, 32767);
+        testSerDes(enc, -100);
+        testSerDes(enc, -10000);
+        testSerDes(enc, -32768);
+        try {
+            testSerDes(enc, 32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[]32768> but was:<[-]32768>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, -32769);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<[-32769]> but was:<[32767]>", e.getMessage());
+        }
+    }
+
+    private void testSerDes(OneMoreByteVLongDimEnc enc, long value) {
+        DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = "" + value;
+        ser.serialize(valueStr, ByteBuffer.wrap(buf));
+        String decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+        Assert.assertEquals(valueStr, decode);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/a6cd4094/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java b/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
new file mode 100644
index 0000000..a1e9516
--- /dev/null
+++ b/core-metadata/src/test/java/org/apache/kylin/dimension/SlimLongDimEncTest.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *  
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.dimension;
+
+import java.nio.ByteBuffer;
+
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SlimLongDimEncTest {
+
+    @Test
+    public void testConstructor() {
+        try {
+            new SlimLongDimEnc(0);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        try {
+            new SlimLongDimEnc(9);
+            Assert.fail();
+        } catch (IllegalArgumentException e) {
+            // expect
+        }
+        new SlimLongDimEnc(8);
+    }
+
+    @Test
+    public void testNull() {
+        for (int i = 1; i < 9; i++) {
+            SlimLongDimEnc enc = new SlimLongDimEnc(i);
+
+            byte[] buf = new byte[enc.getLengthOfEncoding()];
+            enc.encode(null, 0, buf, 0);
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            String decode = enc.decode(buf, 0, buf.length);
+            Assert.assertEquals(null, decode);
+
+            buf = new byte[enc.getLengthOfEncoding()];
+            DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+            ser.serialize(null, ByteBuffer.wrap(buf));
+            Assert.assertTrue(DimensionEncoding.isNull(buf, 0, buf.length));
+            decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+            Assert.assertEquals(null, decode);
+        }
+    }
+
+    @Test
+    public void testEncodeDecode() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(2);
+        testEncodeDecode(enc, 0);
+        testEncodeDecode(enc, 100);
+        testEncodeDecode(enc, 10000);
+        testEncodeDecode(enc, 32767);
+        testEncodeDecode(enc, -100);
+        testEncodeDecode(enc, -10000);
+        testEncodeDecode(enc, -32767);
+        try {
+            testEncodeDecode(enc, 32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<32768> but was:<null>", e.getMessage());
+        }
+        try {
+            testEncodeDecode(enc, -32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<-32768> but was:<null>", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testEncodeDecode2() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(8);
+        testEncodeDecode(enc, 0);
+        testEncodeDecode(enc, 100);
+        testEncodeDecode(enc, 10000);
+        testEncodeDecode(enc, Long.MAX_VALUE);
+        testEncodeDecode(enc, -100);
+        testEncodeDecode(enc, -10000);
+        testEncodeDecode(enc, -Long.MAX_VALUE);
+        try {
+            testEncodeDecode(enc, Long.MIN_VALUE);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<-9223372036854775808> but was:<null>", e.getMessage());
+        }
+    }
+
+    private void testEncodeDecode(SlimLongDimEnc enc, long value) {
+        String valueStr = "" + value;
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        byte[] bytes = Bytes.toBytes(valueStr);
+        enc.encode(bytes, bytes.length, buf, 0);
+        String decode = enc.decode(buf, 0, buf.length);
+        Assert.assertEquals(valueStr, decode);
+    }
+
+    @Test
+    public void testSerDes() {
+        SlimLongDimEnc enc = new SlimLongDimEnc(2);
+        testSerDes(enc, 0);
+        testSerDes(enc, 100);
+        testSerDes(enc, 10000);
+        testSerDes(enc, 32767);
+        testSerDes(enc, -100);
+        testSerDes(enc, -10000);
+        testSerDes(enc, -32767);
+        try {
+            testSerDes(enc, 32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<32768> but was:<null>", e.getMessage());
+        }
+        try {
+            testSerDes(enc, -32768);
+            Assert.fail();
+        } catch (Throwable e) {
+            Assert.assertEquals("expected:<-32768> but was:<null>", e.getMessage());
+        }
+    }
+
+    private void testSerDes(SlimLongDimEnc enc, long value) {
+        DataTypeSerializer<Object> ser = enc.asDataTypeSerializer();
+        byte[] buf = new byte[enc.getLengthOfEncoding()];
+        String valueStr = "" + value;
+        ser.serialize(valueStr, ByteBuffer.wrap(buf));
+        String decode = (String) ser.deserialize(ByteBuffer.wrap(buf));
+        Assert.assertEquals(valueStr, decode);
+    }
+
+}


[09/13] kylin git commit: Revert "rename the streaming_table.json"

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

This reverts commit ffdc5d21d5f46daaf25a0dfed9b3afeef78fc62c.


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

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

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


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

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


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

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

This reverts commit 4e060e7bc89f757a138dfa5beba2a43209e91371.


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

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

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


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

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

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


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

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

This reverts commit 3e081b3fbec4fc8a6cc4ddf8795d2fd581ae04f4.


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

Branch: refs/heads/master
Commit: 96d5f0e0e639fe4e4fc169f687004d9d9361999b
Parents: e604f65
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:49:51 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 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, 49 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/96d5f0e0/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..a5132af 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,9 +65,13 @@ 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/96d5f0e0/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 3b06ed8..fafb1fc 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
@@ -809,8 +809,4 @@ 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/96d5f0e0/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 57b9510..d494fcc 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
@@ -400,8 +400,13 @@ 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 {
 
-        checkBuildingSegment(cube);
+        if (strictChecking)
+            checkNoBuildingSegment(cube);
 
         if (cube.getDescriptor().getModel().getPartitionDesc().isPartitioned()) {
             // try figure out a reasonable start if missing
@@ -431,9 +436,12 @@ public class CubeManager implements IRealizationProvider {
         updateCube(cubeBuilder);
         return newSegment;
     }
-
     public CubeSegment refreshSegment(CubeInstance cube, long startDate, long endDate, long startOffset, long endOffset) throws IOException {
-        checkBuildingSegment(cube);
+        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);
 
@@ -454,7 +462,7 @@ public class CubeManager implements IRealizationProvider {
         if (startDate >= endDate && startOffset >= endOffset)
             throw new IllegalArgumentException("Invalid merge range");
 
-        checkBuildingSegment(cube);
+        checkNoBuildingSegment(cube);
         checkCubeIsPartitioned(cube);
 
         boolean isOffsetsOn = cube.getSegments().get(0).isSourceOffsetsOn();
@@ -580,10 +588,9 @@ public class CubeManager implements IRealizationProvider {
         }
     }
 
-    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; ");
+    private void checkNoBuildingSegment(CubeInstance cube) {
+        if (cube.getBuildingSegments().size() > 0) {
+            throw new IllegalStateException("There is already a building segment!");
         }
     }
 
@@ -722,9 +729,8 @@ public class CubeManager implements IRealizationProvider {
         }
 
         for (CubeSegment seg : tobe) {
-            if (isReady(seg) == false) {
-                logger.warn("For cube " + cube + ", segment " + seg + " isn't READY yet.");
-            }
+            if (isReady(seg) == false)
+                throw new IllegalStateException("For cube " + cube + ", segment " + seg + " should be READY but is not");
         }
 
         List<CubeSegment> toRemoveSegs = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/kylin/blob/96d5f0e0/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 5cae5ac..8808a56 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,7 +207,6 @@ 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/96d5f0e0/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 d42b924..3a19486 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 + ", job id: " + jobId);
+                    throw new IllegalStateTranferException("there is no valid state transfer from:" + oldStatus + " to:" + newStatus);
                 }
                 jobOutput.setStatus(newStatus.toString());
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/96d5f0e0/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 a42ec05..180f0b8 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
@@ -119,7 +119,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);
+            CubeSegment segment = cubeManager.appendSegment(cubeInstance, streamingBatch.getTimeRange().getFirst(), streamingBatch.getTimeRange().getSecond(), 0, 0, false);
             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/96d5f0e0/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..ef10c1e 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,8 +106,7 @@
     }
   } ],
   "override_kylin_properties": {
-    "kylin.cube.algorithm": "inmem",
-    "kylin.cube.building.segment.max": "3"
+    "kylin.cube.algorithm": "inmem"
   },
   "notify_list" : [ ],
   "status_need_notify" : [ ],

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

http://git-wip-us.apache.org/repos/asf/kylin/blob/96d5f0e0/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 d48a473..0000000
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
+++ /dev/null
@@ -1,274 +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.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/96d5f0e0/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/96d5f0e0/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/96d5f0e0/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 9369e6f..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;
@@ -120,7 +125,7 @@ public class SeekOffsetStep extends AbstractExecutable {
             } catch (IOException e) {
                 return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
             }
-            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed, offset start: " + totalStartOffset + ", offset end: " + totalEndOffset);
+            return new ExecuteResult(ExecuteResult.State.SUCCEED, "succeed");
         } else {
             CubeUpdate cubeBuilder = new CubeUpdate(cube);
             cubeBuilder.setToRemoveSegs(segment);


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

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

This reverts commit 1b1b2e37fdcba7ad67f0fa3f2369aa65431f13bc.


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

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

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


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

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

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

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


[04/13] kylin git commit: Revert "change to upper case"

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

This reverts commit aa30880578078369a5844e04a7d7ce736661e902.


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

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

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


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

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


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

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

This reverts commit 1108d9eeccecbccffea0b3f9049151672196c91a.


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

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

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


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

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


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

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

This reverts commit aff2df5987e98ee9fd64d4803a8a2dea90013e40.


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

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

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


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

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

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

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

http://git-wip-us.apache.org/repos/asf/kylin/blob/3ae2549b/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java
----------------------------------------------------------------------
diff --git a/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java b/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java
deleted file mode 100644
index 9362e18..0000000
--- a/core-job/src/test/java/org/apache/kylin/job/DiscardedTestExecutable.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-
-package org.apache.kylin.job;
-
-import org.apache.kylin.job.exception.ExecuteException;
-import org.apache.kylin.job.execution.ExecutableContext;
-import org.apache.kylin.job.execution.ExecuteResult;
-
-/**
- */
-public class DiscardedTestExecutable extends BaseTestExecutable {
-
-    public DiscardedTestExecutable() {
-        super();
-    }
-
-    @Override
-    protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException e) {
-        }
-        return new ExecuteResult(ExecuteResult.State.DISCARDED, "discarded");
-    }
-}

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


[07/13] kylin git commit: Revert "stop scheduler on test finish"

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

This reverts commit 023b5cb1dfdacfa9adcc2586b4aaeb3886f61098.


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

Branch: refs/heads/master
Commit: e604f6527d60b767f8a46a576765cfd205b1efc5
Parents: 8e9c455
Author: Hongbin Ma <ma...@apache.org>
Authored: Mon Sep 19 23:49:28 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Tue Sep 20 11:43:08 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 deletions(-)
----------------------------------------------------------------------


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

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