You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2018/03/28 03:19:42 UTC

[1/5] carbondata git commit: [CARBONDATA-2165]Remove spark in carbon-hadoop module

Repository: carbondata
Updated Branches:
  refs/heads/master 2e1ddb542 -> c723947a7


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
deleted file mode 100644
index bc7b042..0000000
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ /dev/null
@@ -1,236 +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.carbondata.streaming
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.streaming.{CarbonAppendableStreamSink, Sink}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.dictionary.server.{DictionaryServer, NonSecureDictionaryServer}
-import org.apache.carbondata.core.dictionary.service.NonSecureDictionaryServiceProvider
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
-import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
-import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
-import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
-import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
-import org.apache.carbondata.streaming.segment.StreamSegment
-
-/**
- * Stream sink factory
- */
-object StreamSinkFactory {
-
-  def createStreamTableSink(
-      sparkSession: SparkSession,
-      hadoopConf: Configuration,
-      carbonTable: CarbonTable,
-      parameters: Map[String, String]): Sink = {
-    validateParameters(parameters)
-
-    // build load model
-    val carbonLoadModel = buildCarbonLoadModelForStream(
-      sparkSession,
-      hadoopConf,
-      carbonTable,
-      parameters,
-      "")
-    // fire pre event before streamin is started
-    // in case of streaming options and optionsFinal can be same
-    val operationContext = new OperationContext
-    val loadTablePreExecutionEvent = new LoadTablePreExecutionEvent(
-      carbonTable.getCarbonTableIdentifier,
-      carbonLoadModel,
-      carbonLoadModel.getFactFilePath,
-      false,
-      parameters.asJava,
-      parameters.asJava,
-      false
-    )
-    OperationListenerBus.getInstance().fireEvent(loadTablePreExecutionEvent, operationContext)
-    // prepare the stream segment
-    val segmentId = getStreamSegmentId(carbonTable)
-    carbonLoadModel.setSegmentId(segmentId)
-
-    // start server if necessary
-    val server = startDictionaryServer(
-      sparkSession,
-      carbonTable,
-      carbonLoadModel)
-    if (server.isDefined) {
-      carbonLoadModel.setUseOnePass(true)
-    } else {
-      carbonLoadModel.setUseOnePass(false)
-    }
-    // default is carbon appended stream sink
-    val carbonAppendableStreamSink = new CarbonAppendableStreamSink(
-      sparkSession,
-      carbonTable,
-      segmentId,
-      parameters,
-      carbonLoadModel,
-      server)
-
-    // fire post event before streamin is started
-    val loadTablePostExecutionEvent = new LoadTablePostExecutionEvent(
-      carbonTable.getCarbonTableIdentifier,
-      carbonLoadModel
-    )
-    OperationListenerBus.getInstance().fireEvent(loadTablePostExecutionEvent, operationContext)
-    carbonAppendableStreamSink
-  }
-
-  private def validateParameters(parameters: Map[String, String]): Unit = {
-    val segmentSize = parameters.get(CarbonCommonConstants.HANDOFF_SIZE)
-    if (segmentSize.isDefined) {
-      try {
-        val value = java.lang.Long.parseLong(segmentSize.get)
-        if (value < CarbonCommonConstants.HANDOFF_SIZE_MIN) {
-          new CarbonStreamException(CarbonCommonConstants.HANDOFF_SIZE +
-                                    "should be bigger than or equal " +
-                                    CarbonCommonConstants.HANDOFF_SIZE_MIN)
-        }
-      } catch {
-        case _: NumberFormatException =>
-          new CarbonStreamException(CarbonCommonConstants.HANDOFF_SIZE +
-                                    s" $segmentSize is an illegal number")
-      }
-    }
-  }
-
-  /**
-   * get current stream segment id
-   * @return
-   */
-  private def getStreamSegmentId(carbonTable: CarbonTable): String = {
-    val segmentId = StreamSegment.open(carbonTable)
-    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
-    val fileType = FileFactory.getFileType(segmentDir)
-    if (!FileFactory.isFileExist(segmentDir, fileType)) {
-      // Create table directory path, in case of enabling hive metastore first load may not have
-      // table folder created.
-      FileFactory.mkdirs(segmentDir, fileType)
-    }
-    if (FileFactory.isFileExist(segmentDir, fileType)) {
-      // recover fault
-      StreamSegment.recoverSegmentIfRequired(segmentDir)
-    } else {
-      FileFactory.mkdirs(segmentDir, fileType)
-    }
-    segmentId
-  }
-
-  def startDictionaryServer(
-      sparkSession: SparkSession,
-      carbonTable: CarbonTable,
-      carbonLoadModel: CarbonLoadModel): Option[DictionaryServer] = {
-    // start dictionary server when use one pass load and dimension with DICTIONARY
-    // encoding is present.
-    val allDimensions = carbonTable.getAllDimensions.asScala.toList
-    val createDictionary = allDimensions.exists {
-      carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
-                         !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
-    }
-    val carbonSecureModeDictServer = CarbonProperties.getInstance.
-      getProperty(CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER,
-        CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER_DEFAULT)
-
-    val sparkConf = sparkSession.sqlContext.sparkContext.getConf
-    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
-      getConf.get("spark.driver.host")
-
-    val server: Option[DictionaryServer] = if (createDictionary) {
-      if (sparkConf.get("spark.authenticate", "false").equalsIgnoreCase("true") &&
-          carbonSecureModeDictServer.toBoolean) {
-        val dictionaryServer = SecureDictionaryServer.getInstance(sparkConf,
-          sparkDriverHost.toString, carbonLoadModel.getDictionaryServerPort, carbonTable)
-        carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-        carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
-        carbonLoadModel.setDictionaryServerSecretKey(dictionaryServer.getSecretKey)
-        carbonLoadModel.setDictionaryEncryptServerSecure(dictionaryServer.isEncryptSecureServer)
-        carbonLoadModel.setDictionaryServiceProvider(new SecureDictionaryServiceProvider())
-        sparkSession.sparkContext.addSparkListener(new SparkListener() {
-          override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-            dictionaryServer.shutdown()
-          }
-        })
-        Some(dictionaryServer)
-      } else {
-        val dictionaryServer = NonSecureDictionaryServer
-          .getInstance(carbonLoadModel.getDictionaryServerPort, carbonTable)
-        carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-        carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
-        carbonLoadModel.setDictionaryEncryptServerSecure(false)
-        carbonLoadModel
-          .setDictionaryServiceProvider(new NonSecureDictionaryServiceProvider(dictionaryServer
-            .getPort))
-        sparkSession.sparkContext.addSparkListener(new SparkListener() {
-          override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-            dictionaryServer.shutdown()
-          }
-        })
-        Some(dictionaryServer)
-      }
-    } else {
-      None
-    }
-    server
-  }
-
-  private def buildCarbonLoadModelForStream(
-      sparkSession: SparkSession,
-      hadoopConf: Configuration,
-      carbonTable: CarbonTable,
-      parameters: Map[String, String],
-      segmentId: String): CarbonLoadModel = {
-    val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
-    carbonProperty.addProperty("zookeeper.enable.lock", "false")
-    val optionsFinal = LoadOption.fillOptionWithDefaultValue(parameters.asJava)
-    optionsFinal.put("sort_scope", "no_sort")
-    if (parameters.get("fileheader").isEmpty) {
-      optionsFinal.put("fileheader", carbonTable.getCreateOrderColumn(carbonTable.getTableName)
-        .asScala.map(_.getColName).mkString(","))
-    }
-    val carbonLoadModel = new CarbonLoadModel()
-    new CarbonLoadModelBuilder(carbonTable).build(
-      parameters.asJava,
-      optionsFinal,
-      carbonLoadModel,
-      hadoopConf)
-    carbonLoadModel.setSegmentId(segmentId)
-    // stream should use one pass
-    val dictionaryServerPort = parameters.getOrElse(
-      CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-      carbonProperty.getProperty(
-        CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-        CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT))
-    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
-      getConf.get("spark.driver.host")
-    carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
-    carbonLoadModel.setDictionaryServerPort(dictionaryServerPort.toInt)
-    carbonLoadModel
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala
new file mode 100644
index 0000000..8661417
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/parser/FieldConverter.scala
@@ -0,0 +1,95 @@
+/*
+ * 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.carbondata.streaming.parser
+
+import java.nio.charset.Charset
+import java.text.SimpleDateFormat
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
+object FieldConverter {
+
+  /**
+   * Return a String representation of the input value
+   * @param value input value
+   * @param serializationNullFormat string for null value
+   * @param delimiterLevel1 level 1 delimiter for complex type
+   * @param delimiterLevel2 level 2 delimiter for complex type
+   * @param timeStampFormat timestamp format
+   * @param dateFormat date format
+   * @param level level for recursive call
+   */
+  def objectToString(
+      value: Any,
+      serializationNullFormat: String,
+      delimiterLevel1: String,
+      delimiterLevel2: String,
+      timeStampFormat: SimpleDateFormat,
+      dateFormat: SimpleDateFormat,
+      level: Int = 1): String = {
+    if (value == null) {
+      serializationNullFormat
+    } else {
+      value match {
+        case s: String => if (s.length > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
+          throw new Exception("Dataload failed, String length cannot exceed " +
+                              CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " characters")
+        } else {
+          s
+        }
+        case d: java.math.BigDecimal => d.toPlainString
+        case i: java.lang.Integer => i.toString
+        case d: java.lang.Double => d.toString
+        case t: java.sql.Timestamp => timeStampFormat format t
+        case d: java.sql.Date => dateFormat format d
+        case b: java.lang.Boolean => b.toString
+        case s: java.lang.Short => s.toString
+        case f: java.lang.Float => f.toString
+        case bs: Array[Byte] => new String(bs,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET))
+        case s: scala.collection.Seq[Any] =>
+          val delimiter = if (level == 1) {
+            delimiterLevel1
+          } else {
+            delimiterLevel2
+          }
+          val builder = new StringBuilder()
+          s.foreach { x =>
+            builder.append(objectToString(x, serializationNullFormat, delimiterLevel1,
+              delimiterLevel2, timeStampFormat, dateFormat, level + 1)).append(delimiter)
+          }
+          builder.substring(0, builder.length - delimiter.length())
+        case m: scala.collection.Map[Any, Any] =>
+          throw new Exception("Unsupported data type: Map")
+        case r: org.apache.spark.sql.Row =>
+          val delimiter = if (level == 1) {
+            delimiterLevel1
+          } else {
+            delimiterLevel2
+          }
+          val builder = new StringBuilder()
+          for (i <- 0 until r.length) {
+            builder.append(objectToString(r(i), serializationNullFormat, delimiterLevel1,
+              delimiterLevel2, timeStampFormat, dateFormat, level + 1)).append(delimiter)
+          }
+          builder.substring(0, builder.length - delimiter.length())
+        case other => other.toString
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala
index 5a227cf..1696fdc 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.streaming.parser
 
+import java.nio.charset.Charset
 import java.text.SimpleDateFormat
 
 import org.apache.hadoop.conf.Configuration
@@ -27,7 +28,6 @@ import org.apache.spark.sql.types.StructType
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
-import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 /**
  * SparkSQL Row Stream Parser.
@@ -61,12 +61,13 @@ class RowStreamParserImp extends CarbonStreamParser {
 
   override def parserRow(value: InternalRow): Array[Object] = {
     this.encoder.fromRow(value).toSeq.map { x => {
-      CarbonScalaUtil.getString(x,
-        serializationNullFormat, complexDelimiterLevel1, complexDelimiterLevel2,
+      FieldConverter.objectToString(
+        x, serializationNullFormat, complexDelimiterLevel1, complexDelimiterLevel2,
         timeStampFormat, dateFormat)
     } }.toArray
   }
 
   override def close(): Unit = {
   }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
deleted file mode 100644
index 6e6d092..0000000
--- a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ /dev/null
@@ -1,362 +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.spark.sql.execution.streaming
-
-import java.util.Date
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce._
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
-import org.apache.spark.TaskContext
-import org.apache.spark.internal.io.FileCommitProtocol
-import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
-import org.apache.spark.sql.{DataFrame, SparkSession}
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.execution.{QueryExecution, SQLExecution}
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.util.{SerializableConfiguration, Utils}
-
-import org.apache.carbondata.common.CarbonIterator
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.dictionary.server.DictionaryServer
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.stats.QueryStatistic
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
-import org.apache.carbondata.hadoop.streaming.CarbonStreamOutputFormat
-import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
-import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.streaming.{CarbonStreamException, StreamHandoffRDD}
-import org.apache.carbondata.streaming.parser.CarbonStreamParser
-import org.apache.carbondata.streaming.segment.StreamSegment
-
-/**
- * an implement of stream sink, it persist each batch to disk by appending the batch data to
- * data files.
- */
-class CarbonAppendableStreamSink(
-    sparkSession: SparkSession,
-    val carbonTable: CarbonTable,
-    var currentSegmentId: String,
-    parameters: Map[String, String],
-    carbonLoadModel: CarbonLoadModel,
-    server: Option[DictionaryServer]) extends Sink {
-
-  private val fileLogPath = CarbonTablePath.getStreamingLogDir(carbonTable.getTablePath)
-  private val fileLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, fileLogPath)
-  // prepare configuration
-  private val hadoopConf = {
-    val conf = sparkSession.sessionState.newHadoopConf()
-    // put all parameters into hadoopConf
-    parameters.foreach { entry =>
-      conf.set(entry._1, entry._2)
-    }
-    // properties below will be used for default CarbonStreamParser
-    conf.set("carbon_complex_delimiter_level_1",
-      carbonLoadModel.getComplexDelimiterLevel1)
-    conf.set("carbon_complex_delimiter_level_2",
-      carbonLoadModel.getComplexDelimiterLevel2)
-    conf.set(
-      DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT,
-      carbonLoadModel.getSerializationNullFormat().split(",")(1))
-    conf.set(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-      carbonLoadModel.getTimestampformat())
-    conf.set(
-      CarbonCommonConstants.CARBON_DATE_FORMAT,
-      carbonLoadModel.getDateFormat())
-    conf
-  }
-  // segment max size(byte)
-  private val segmentMaxSize = hadoopConf.getLong(
-    CarbonCommonConstants.HANDOFF_SIZE,
-    CarbonProperties.getInstance().getHandoffSize
-  )
-
-  // auto handoff
-  private val enableAutoHandoff = hadoopConf.getBoolean(
-    CarbonCommonConstants.ENABLE_AUTO_HANDOFF,
-    CarbonProperties.getInstance().isEnableAutoHandoff
-  )
-
-  override def addBatch(batchId: Long, data: DataFrame): Unit = {
-    if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) {
-      CarbonAppendableStreamSink.LOGGER.info(s"Skipping already committed batch $batchId")
-    } else {
-
-      val statistic = new QueryStatistic()
-
-      // fire pre event on every batch add
-      // in case of streaming options and optionsFinal can be same
-      val operationContext = new OperationContext
-      val loadTablePreExecutionEvent = new LoadTablePreExecutionEvent(
-        carbonTable.getCarbonTableIdentifier,
-        carbonLoadModel,
-        carbonLoadModel.getFactFilePath,
-        false,
-        parameters.asJava,
-        parameters.asJava,
-        false
-      )
-      OperationListenerBus.getInstance().fireEvent(loadTablePreExecutionEvent, operationContext)
-      checkOrHandOffSegment()
-
-      // committer will record how this spark job commit its output
-      val committer = FileCommitProtocol.instantiate(
-        className = sparkSession.sessionState.conf.streamingFileCommitProtocolClass,
-        jobId = batchId.toString,
-        outputPath = fileLogPath,
-        isAppend = false)
-
-      committer match {
-        case manifestCommitter: ManifestFileCommitProtocol =>
-          manifestCommitter.setupManifestOptions(fileLog, batchId)
-        case _ => // Do nothing
-      }
-
-      CarbonAppendableStreamSink.writeDataFileJob(
-        sparkSession,
-        carbonTable,
-        parameters,
-        batchId,
-        currentSegmentId,
-        data.queryExecution,
-        committer,
-        hadoopConf,
-        carbonLoadModel,
-        server)
-      // fire post event on every batch add
-      val loadTablePostExecutionEvent = new LoadTablePostExecutionEvent(
-        carbonTable.getCarbonTableIdentifier,
-        carbonLoadModel
-      )
-      OperationListenerBus.getInstance().fireEvent(loadTablePostExecutionEvent, operationContext)
-
-      statistic.addStatistics(s"add batch: $batchId", System.currentTimeMillis())
-      CarbonAppendableStreamSink.LOGGER.info(
-        s"${statistic.getMessage}, taken time(ms): ${statistic.getTimeTaken}")
-    }
-  }
-
-  /**
-   * if the directory size of current segment beyond the threshold, hand off new segment
-   */
-  private def checkOrHandOffSegment(): Unit = {
-    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
-    val fileType = FileFactory.getFileType(segmentDir)
-    if (segmentMaxSize <= StreamSegment.size(segmentDir)) {
-      val newSegmentId = StreamSegment.close(carbonTable, currentSegmentId)
-      currentSegmentId = newSegmentId
-      val newSegmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
-      FileFactory.mkdirs(newSegmentDir, fileType)
-
-      // TODO trigger hand off operation
-      if (enableAutoHandoff) {
-        StreamHandoffRDD.startStreamingHandoffThread(
-          carbonLoadModel,
-          sparkSession,
-          false)
-      }
-    }
-  }
-}
-
-object CarbonAppendableStreamSink {
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  /**
-   * package the hadoop configuration and it will be passed to executor side from driver side
-   */
-  case class WriteDataFileJobDescription(
-      serializableHadoopConf: SerializableConfiguration,
-      batchId: Long,
-      segmentId: String)
-
-  /**
-   * Run a spark job to append the newly arrived data to the existing row format
-   * file directly.
-   * If there are failure in the task, spark will re-try the task and
-   * carbon will do recovery by HDFS file truncate. (see StreamSegment.tryRecoverFromTaskFault)
-   * If there are job level failure, every files in the stream segment will do truncate
-   * if necessary. (see StreamSegment.tryRecoverFromJobFault)
-   */
-  def writeDataFileJob(
-      sparkSession: SparkSession,
-      carbonTable: CarbonTable,
-      parameters: Map[String, String],
-      batchId: Long,
-      segmentId: String,
-      queryExecution: QueryExecution,
-      committer: FileCommitProtocol,
-      hadoopConf: Configuration,
-      carbonLoadModel: CarbonLoadModel,
-      server: Option[DictionaryServer]): Unit = {
-
-    // create job
-    val job = Job.getInstance(hadoopConf)
-    job.setOutputKeyClass(classOf[Void])
-    job.setOutputValueClass(classOf[InternalRow])
-    val jobId = CarbonInputFormatUtil.getJobId(new Date, batchId.toInt)
-    job.setJobID(jobId)
-
-    val description = WriteDataFileJobDescription(
-      serializableHadoopConf = new SerializableConfiguration(job.getConfiguration),
-      batchId,
-      segmentId
-    )
-
-    // run write data file job
-    SQLExecution.withNewExecutionId(sparkSession, queryExecution) {
-      var result: Array[TaskCommitMessage] = null
-      try {
-        committer.setupJob(job)
-        // initialize dictionary server
-        if (server.isDefined) {
-          server.get.initializeDictionaryGenerator(carbonTable)
-        }
-
-        val rowSchema = queryExecution.analyzed.schema
-        // write data file
-        result = sparkSession.sparkContext.runJob(queryExecution.toRdd,
-          (taskContext: TaskContext, iterator: Iterator[InternalRow]) => {
-            writeDataFileTask(
-              description,
-              carbonLoadModel,
-              sparkStageId = taskContext.stageId(),
-              sparkPartitionId = taskContext.partitionId(),
-              sparkAttemptNumber = taskContext.attemptNumber(),
-              committer,
-              iterator,
-              rowSchema
-            )
-          })
-
-        // write dictionary
-        if (server.isDefined) {
-          try {
-            server.get.writeTableDictionary(carbonTable.getCarbonTableIdentifier.getTableId)
-          } catch {
-            case _: Exception =>
-              LOGGER.error(
-                s"Error while writing dictionary file for ${carbonTable.getTableUniqueName}")
-              throw new Exception(
-                "Streaming ingest failed due to error while writing dictionary file")
-          }
-        }
-
-        // update data file info in index file
-        StreamSegment.updateIndexFile(
-          CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId))
-
-      } catch {
-        // catch fault of executor side
-        case t: Throwable =>
-          val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
-          StreamSegment.recoverSegmentIfRequired(segmentDir)
-          LOGGER.error(t, s"Aborting job ${ job.getJobID }.")
-          committer.abortJob(job)
-          throw new CarbonStreamException("Job failed to write data file", t)
-      }
-      committer.commitJob(job, result)
-      LOGGER.info(s"Job ${ job.getJobID } committed.")
-    }
-  }
-
-  /**
-   * execute a task for each partition to write a data file
-   */
-  def writeDataFileTask(
-      description: WriteDataFileJobDescription,
-      carbonLoadModel: CarbonLoadModel,
-      sparkStageId: Int,
-      sparkPartitionId: Int,
-      sparkAttemptNumber: Int,
-      committer: FileCommitProtocol,
-      iterator: Iterator[InternalRow],
-      rowSchema: StructType
-  ): TaskCommitMessage = {
-
-    val jobId = CarbonInputFormatUtil.getJobId(new Date, sparkStageId)
-    val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId)
-    val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber)
-
-    // Set up the attempt context required to use in the output committer.
-    val taskAttemptContext: TaskAttemptContext = {
-      // Set up the configuration object
-      val hadoopConf = description.serializableHadoopConf.value
-      CarbonStreamOutputFormat.setSegmentId(hadoopConf, description.segmentId)
-      hadoopConf.set("mapred.job.id", jobId.toString)
-      hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID.toString)
-      hadoopConf.set("mapred.task.id", taskAttemptId.toString)
-      hadoopConf.setBoolean("mapred.task.is.map", true)
-      hadoopConf.setInt("mapred.task.partition", 0)
-      new TaskAttemptContextImpl(hadoopConf, taskAttemptId)
-    }
-
-    committer.setupTask(taskAttemptContext)
-
-    try {
-      Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
-
-        val parserName = taskAttemptContext.getConfiguration.get(
-          CarbonStreamParser.CARBON_STREAM_PARSER,
-          CarbonStreamParser.CARBON_STREAM_PARSER_DEFAULT)
-
-        val streamParser =
-          Class.forName(parserName).newInstance.asInstanceOf[CarbonStreamParser]
-        streamParser.initialize(taskAttemptContext.getConfiguration, rowSchema)
-
-        StreamSegment.appendBatchData(new InputIterator(iterator, streamParser),
-          taskAttemptContext, carbonLoadModel)
-      })(catchBlock = {
-        committer.abortTask(taskAttemptContext)
-        LOGGER.error(s"Job $jobId aborted.")
-      })
-      committer.commitTask(taskAttemptContext)
-    } catch {
-      case t: Throwable =>
-        throw new CarbonStreamException("Task failed while writing rows", t)
-    }
-  }
-
-  /**
-   * convert spark iterator to carbon iterator, so that java module can use it.
-   */
-  class InputIterator(rddIter: Iterator[InternalRow], streamParser: CarbonStreamParser)
-    extends CarbonIterator[Array[Object]] {
-
-    override def hasNext: Boolean = rddIter.hasNext
-
-    override def next: Array[Object] = {
-      streamParser.parserRow(rddIter.next())
-    }
-
-    override def close(): Unit = {
-      streamParser.close()
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
deleted file mode 100644
index 2f911c5..0000000
--- a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonStreamingQueryListener.scala
+++ /dev/null
@@ -1,77 +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.spark.sql.execution.streaming
-
-import java.util
-import java.util.UUID
-
-import org.apache.spark.SPARK_VERSION
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.streaming.StreamingQueryListener
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-
-class CarbonStreamingQueryListener(spark: SparkSession) extends StreamingQueryListener {
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  private val cache = new util.HashMap[UUID, ICarbonLock]()
-
-  override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {
-    val streamQuery = spark.streams.get(event.id)
-    val qry = if (streamQuery.isInstanceOf[StreamExecution]) {
-      // adapt spark 2.1
-      streamQuery.asInstanceOf[StreamExecution]
-    } else {
-      // adapt spark 2.2 and later version
-      val clazz = Class.forName("org.apache.spark.sql.execution.streaming.StreamingQueryWrapper")
-      val method = clazz.getMethod("streamingQuery")
-      method.invoke(streamQuery).asInstanceOf[StreamExecution]
-    }
-    if (qry.sink.isInstanceOf[CarbonAppendableStreamSink]) {
-      LOGGER.info("Carbon streaming query started: " + event.id)
-      val sink = qry.sink.asInstanceOf[CarbonAppendableStreamSink]
-      val carbonTable = sink.carbonTable
-      val lock = CarbonLockFactory.getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
-        LockUsage.STREAMING_LOCK)
-      if (lock.lockWithRetries()) {
-        LOGGER.info("Acquired the lock for stream table: " + carbonTable.getDatabaseName + "." +
-                    carbonTable.getTableName)
-        cache.put(event.id, lock)
-      } else {
-        LOGGER.error("Not able to acquire the lock for stream table:" +
-                     carbonTable.getDatabaseName + "." + carbonTable.getTableName)
-        throw new InterruptedException(
-          "Not able to acquire the lock for stream table: " + carbonTable.getDatabaseName + "." +
-          carbonTable.getTableName)
-      }
-    }
-  }
-
-  override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {
-  }
-
-  override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {
-    val lock = cache.remove(event.id)
-    if (null != lock) {
-      LOGGER.info("Carbon streaming query: " + event.id)
-      lock.unlock()
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamInputFormatTest.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamInputFormatTest.java b/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamInputFormatTest.java
new file mode 100644
index 0000000..a224446
--- /dev/null
+++ b/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamInputFormatTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.carbondata.streaming;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.statusmanager.FileFormat;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CarbonStreamInputFormatTest extends TestCase {
+
+  private TaskAttemptID taskAttemptId;
+  private TaskAttemptContext taskAttemptContext;
+  private Configuration hadoopConf;
+  private AbsoluteTableIdentifier identifier;
+  private String tablePath;
+
+
+  @Override protected void setUp() throws Exception {
+    tablePath = new File("target/stream_input").getCanonicalPath();
+    String dbName = "default";
+    String tableName = "stream_table_input";
+    identifier = AbsoluteTableIdentifier.from(
+        tablePath,
+        new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+
+    JobID jobId = CarbonInputFormatUtil.getJobId(new Date(), 0);
+    TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
+    taskAttemptId = new TaskAttemptID(taskId, 0);
+
+    hadoopConf = new Configuration();
+    taskAttemptContext = new TaskAttemptContextImpl(hadoopConf, taskAttemptId);
+  }
+
+  private InputSplit buildInputSplit() throws IOException {
+    CarbonInputSplit carbonInputSplit = new CarbonInputSplit();
+    List<CarbonInputSplit> splitList = new ArrayList<>();
+    splitList.add(carbonInputSplit);
+    return new CarbonMultiBlockSplit(splitList, new String[] { "localhost" },
+        FileFormat.ROW_V1);
+  }
+
+  @Test public void testCreateRecordReader() {
+    try {
+      InputSplit inputSplit = buildInputSplit();
+      CarbonStreamInputFormat inputFormat = new CarbonStreamInputFormat();
+      RecordReader recordReader = inputFormat.createRecordReader(inputSplit, taskAttemptContext);
+      Assert.assertNotNull("Failed to create record reader", recordReader);
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.assertTrue(e.getMessage(), false);
+    }
+  }
+
+  @Override protected void tearDown() throws Exception {
+    super.tearDown();
+    if (tablePath != null) {
+      FileFactory.deleteAllFilesOfDir(new File(tablePath));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamOutputFormatTest.java b/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamOutputFormatTest.java
new file mode 100644
index 0000000..af79483
--- /dev/null
+++ b/streaming/src/test/java/org/apache/carbondata/streaming/CarbonStreamOutputFormatTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.carbondata.streaming;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Date;
+import java.util.UUID;
+
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.hadoop.testutil.StoreCreator;
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CarbonStreamOutputFormatTest extends TestCase {
+
+  private Configuration hadoopConf;
+  private TaskAttemptID taskAttemptId;
+  private CarbonLoadModel carbonLoadModel;
+  private String tablePath;
+
+  @Override protected void setUp() throws Exception {
+    super.setUp();
+    JobID jobId = CarbonInputFormatUtil.getJobId(new Date(), 0);
+    TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
+    taskAttemptId = new TaskAttemptID(taskId, 0);
+
+    hadoopConf = new Configuration();
+    hadoopConf.set("mapred.job.id", jobId.toString());
+    hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID().toString());
+    hadoopConf.set("mapred.task.id", taskAttemptId.toString());
+    hadoopConf.setBoolean("mapred.task.is.map", true);
+    hadoopConf.setInt("mapred.task.partition", 0);
+
+    tablePath = new File("target/stream_output").getCanonicalPath();
+    String dbName = "default";
+    String tableName = "stream_table_output";
+    AbsoluteTableIdentifier identifier =
+        AbsoluteTableIdentifier.from(
+            tablePath,
+            new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+
+    CarbonTable table = StoreCreator.createTable(identifier);
+
+    String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
+    carbonLoadModel = StoreCreator.buildCarbonLoadModel(table, factFilePath, identifier);
+  }
+
+  @Test public void testSetCarbonLoadModel() {
+    try {
+      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
+    } catch (IOException e) {
+      Assert.assertTrue("Failed to config CarbonLoadModel for CarbonStreamOutputFromat", false);
+    }
+  }
+
+  @Test public void testGetCarbonLoadModel() {
+    try {
+      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
+      CarbonLoadModel model = CarbonStreamOutputFormat.getCarbonLoadModel(hadoopConf);
+
+      Assert.assertNotNull("Failed to get CarbonLoadModel", model);
+      Assert.assertTrue("CarbonLoadModel should be same with previous",
+          carbonLoadModel.getFactTimeStamp() == model.getFactTimeStamp());
+
+    } catch (IOException e) {
+      Assert.assertTrue("Failed to get CarbonLoadModel for CarbonStreamOutputFromat", false);
+    }
+  }
+
+  @Test public void testGetRecordWriter() {
+    CarbonStreamOutputFormat outputFormat = new CarbonStreamOutputFormat();
+    try {
+      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
+      TaskAttemptContext taskAttemptContext =
+          new TaskAttemptContextImpl(hadoopConf, taskAttemptId);
+      RecordWriter recordWriter = outputFormat.getRecordWriter(taskAttemptContext);
+      Assert.assertNotNull("Failed to get CarbonStreamRecordWriter", recordWriter);
+    } catch (Exception e) {
+      e.printStackTrace();
+      Assert.assertTrue(e.getMessage(), false);
+    }
+  }
+
+  @Override protected void tearDown() throws Exception {
+    super.tearDown();
+    if (tablePath != null) {
+      FileFactory.deleteAllFilesOfDir(new File(tablePath));
+    }
+  }
+}


[4/5] carbondata git commit: [CARBONDATA-2165]Remove spark in carbon-hadoop module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
new file mode 100644
index 0000000..b2c2d39
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
@@ -0,0 +1,495 @@
+/*
+ * 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.carbondata.hadoop.testutil;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.DataMapStoreManager;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnIdentifier;
+import org.apache.carbondata.core.metadata.converter.SchemaConverter;
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
+import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.TableSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
+import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
+import org.apache.carbondata.core.writer.ThriftWriter;
+import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
+import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
+import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo;
+import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfoPreparator;
+import org.apache.carbondata.processing.loading.DataLoadExecutor;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.loading.csvinput.BlockDetails;
+import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
+import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator;
+import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable;
+import org.apache.carbondata.processing.loading.model.CarbonDataLoadSchema;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.util.TableOptionConstant;
+
+import com.google.gson.Gson;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+/**
+ * This class will create store file based on provided schema
+ *
+ */
+public class StoreCreator {
+
+  private static LogService LOG =
+      LogServiceFactory.getLogService(StoreCreator.class.getCanonicalName());
+  private static AbsoluteTableIdentifier absoluteTableIdentifier;
+  private static String storePath = null;
+
+  static {
+    storePath = new File("target/store").getAbsolutePath();
+    String dbName = "testdb";
+    String tableName = "testtable";
+    absoluteTableIdentifier = AbsoluteTableIdentifier.from(storePath + "/testdb/testtable",
+        new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+  }
+
+  public static AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
+    return absoluteTableIdentifier;
+  }
+
+  public static CarbonLoadModel buildCarbonLoadModel(CarbonTable table, String factFilePath,
+      AbsoluteTableIdentifier absoluteTableIdentifier) {
+    CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
+    CarbonLoadModel loadModel = new CarbonLoadModel();
+    loadModel.setCarbonDataLoadSchema(schema);
+    loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+    loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    loadModel.setFactFilePath(factFilePath);
+    loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
+    loadModel.setTablePath(absoluteTableIdentifier.getTablePath());
+    loadModel.setDateFormat(null);
+    loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
+        CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS));
+    loadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty(
+        CarbonCommonConstants.CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
+    loadModel
+        .setSerializationNullFormat(
+            TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N");
+    loadModel
+        .setBadRecordsLoggerEnable(
+            TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false");
+    loadModel
+        .setBadRecordsAction(
+            TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "FORCE");
+    loadModel
+        .setIsEmptyDataBadRecord(
+            DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + "false");
+    loadModel.setCsvHeader("ID,date,country,name,phonetype,serialname,salary");
+    loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(","));
+    loadModel.setTaskNo("0");
+    loadModel.setSegmentId("0");
+    loadModel.setFactTimeStamp(System.currentTimeMillis());
+    loadModel.setMaxColumns("10");
+    return loadModel;
+  }
+
+  /**
+   * Create store without any restructure
+   */
+  public static void createCarbonStore() throws Exception {
+    CarbonLoadModel loadModel = createTableAndLoadModel();
+    loadData(loadModel, storePath);
+  }
+
+  /**
+   * Method to clear the data maps
+   */
+  public static void clearDataMaps() {
+    DataMapStoreManager.getInstance().clearDataMaps(absoluteTableIdentifier);
+  }
+
+  public static CarbonLoadModel createTableAndLoadModel() throws Exception {
+    String factFilePath =
+        new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
+    File storeDir = new File(storePath);
+    CarbonUtil.deleteFoldersAndFiles(storeDir);
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS,
+        storePath);
+
+    CarbonTable table = createTable(absoluteTableIdentifier);
+    writeDictionary(factFilePath, table);
+    return buildCarbonLoadModel(table, factFilePath, absoluteTableIdentifier);
+  }
+
+  public static CarbonTable createTable(
+      AbsoluteTableIdentifier identifier) throws IOException {
+    TableInfo tableInfo = new TableInfo();
+    tableInfo.setDatabaseName(identifier.getCarbonTableIdentifier().getDatabaseName());
+    TableSchema tableSchema = new TableSchema();
+    tableSchema.setTableName(identifier.getCarbonTableIdentifier().getTableName());
+    List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
+    ArrayList<Encoding> encodings = new ArrayList<>();
+    encodings.add(Encoding.DICTIONARY);
+    ColumnSchema id = new ColumnSchema();
+    id.setColumnName("ID");
+    id.setColumnar(true);
+    id.setDataType(DataTypes.INT);
+    id.setEncodingList(encodings);
+    id.setColumnUniqueId(UUID.randomUUID().toString());
+    id.setColumnReferenceId(id.getColumnUniqueId());
+    id.setDimensionColumn(true);
+    id.setColumnGroup(1);
+    columnSchemas.add(id);
+
+    ColumnSchema date = new ColumnSchema();
+    date.setColumnName("date");
+    date.setColumnar(true);
+    date.setDataType(DataTypes.STRING);
+    date.setEncodingList(encodings);
+    date.setColumnUniqueId(UUID.randomUUID().toString());
+    date.setDimensionColumn(true);
+    date.setColumnGroup(2);
+    date.setSortColumn(true);
+    date.setColumnReferenceId(id.getColumnUniqueId());
+    columnSchemas.add(date);
+
+    ColumnSchema country = new ColumnSchema();
+    country.setColumnName("country");
+    country.setColumnar(true);
+    country.setDataType(DataTypes.STRING);
+    country.setEncodingList(encodings);
+    country.setColumnUniqueId(UUID.randomUUID().toString());
+    country.setDimensionColumn(true);
+    country.setColumnGroup(3);
+    country.setSortColumn(true);
+    country.setColumnReferenceId(id.getColumnUniqueId());
+    columnSchemas.add(country);
+
+    ColumnSchema name = new ColumnSchema();
+    name.setColumnName("name");
+    name.setColumnar(true);
+    name.setDataType(DataTypes.STRING);
+    name.setEncodingList(encodings);
+    name.setColumnUniqueId(UUID.randomUUID().toString());
+    name.setDimensionColumn(true);
+    name.setColumnGroup(4);
+    name.setSortColumn(true);
+    name.setColumnReferenceId(id.getColumnUniqueId());
+    columnSchemas.add(name);
+
+    ColumnSchema phonetype = new ColumnSchema();
+    phonetype.setColumnName("phonetype");
+    phonetype.setColumnar(true);
+    phonetype.setDataType(DataTypes.STRING);
+    phonetype.setEncodingList(encodings);
+    phonetype.setColumnUniqueId(UUID.randomUUID().toString());
+    phonetype.setDimensionColumn(true);
+    phonetype.setColumnGroup(5);
+    phonetype.setSortColumn(true);
+    phonetype.setColumnReferenceId(id.getColumnUniqueId());
+    columnSchemas.add(phonetype);
+
+    ColumnSchema serialname = new ColumnSchema();
+    serialname.setColumnName("serialname");
+    serialname.setColumnar(true);
+    serialname.setDataType(DataTypes.STRING);
+    serialname.setEncodingList(encodings);
+    serialname.setColumnUniqueId(UUID.randomUUID().toString());
+    serialname.setDimensionColumn(true);
+    serialname.setColumnGroup(6);
+    serialname.setSortColumn(true);
+    serialname.setColumnReferenceId(id.getColumnUniqueId());
+    columnSchemas.add(serialname);
+
+    ColumnSchema salary = new ColumnSchema();
+    salary.setColumnName("salary");
+    salary.setColumnar(true);
+    salary.setDataType(DataTypes.INT);
+    salary.setEncodingList(new ArrayList<Encoding>());
+    salary.setColumnUniqueId(UUID.randomUUID().toString());
+    salary.setDimensionColumn(false);
+    salary.setColumnReferenceId(id.getColumnUniqueId());
+    salary.setColumnGroup(7);
+    columnSchemas.add(salary);
+
+    tableSchema.setListOfColumns(columnSchemas);
+    SchemaEvolution schemaEvol = new SchemaEvolution();
+    schemaEvol.setSchemaEvolutionEntryList(new ArrayList<SchemaEvolutionEntry>());
+    tableSchema.setSchemaEvalution(schemaEvol);
+    tableSchema.setTableId(UUID.randomUUID().toString());
+    tableInfo.setTableUniqueName(
+        identifier.getCarbonTableIdentifier().getTableUniqueName()
+    );
+    tableInfo.setLastUpdatedTime(System.currentTimeMillis());
+    tableInfo.setFactTable(tableSchema);
+    tableInfo.setTablePath(identifier.getTablePath());
+    String schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath());
+    String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
+    CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
+
+    SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
+    org.apache.carbondata.format.TableInfo thriftTableInfo =
+        schemaConverter.fromWrapperToExternalTableInfo(
+            tableInfo,
+            tableInfo.getDatabaseName(),
+            tableInfo.getFactTable().getTableName());
+    org.apache.carbondata.format.SchemaEvolutionEntry schemaEvolutionEntry =
+        new org.apache.carbondata.format.SchemaEvolutionEntry(tableInfo.getLastUpdatedTime());
+    thriftTableInfo.getFact_table().getSchema_evolution().getSchema_evolution_history()
+        .add(schemaEvolutionEntry);
+
+    FileFactory.FileType fileType = FileFactory.getFileType(schemaMetadataPath);
+    if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
+      FileFactory.mkdirs(schemaMetadataPath, fileType);
+    }
+
+    ThriftWriter thriftWriter = new ThriftWriter(schemaFilePath, false);
+    thriftWriter.open();
+    thriftWriter.write(thriftTableInfo);
+    thriftWriter.close();
+    return CarbonMetadata.getInstance().getCarbonTable(tableInfo.getTableUniqueName());
+  }
+
+  private static void writeDictionary(String factFilePath, CarbonTable table) throws Exception {
+    BufferedReader reader = new BufferedReader(new InputStreamReader(
+        new FileInputStream(factFilePath), "UTF-8"));
+    List<CarbonDimension> dims = table.getDimensionByTableName(table.getTableName());
+    Set<String>[] set = new HashSet[dims.size()];
+    for (int i = 0; i < set.length; i++) {
+      set[i] = new HashSet<String>();
+    }
+    String line = reader.readLine();
+    while (line != null) {
+      String[] data = line.split(",");
+      for (int i = 0; i < set.length; i++) {
+        set[i].add(data[i]);
+      }
+      line = reader.readLine();
+    }
+
+    Cache dictCache = CacheProvider.getInstance()
+        .createCache(CacheType.REVERSE_DICTIONARY);
+    for (int i = 0; i < set.length; i++) {
+      ColumnIdentifier columnIdentifier =
+          new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
+          new DictionaryColumnUniqueIdentifier(
+              table.getAbsoluteTableIdentifier(), columnIdentifier, columnIdentifier.getDataType());
+      CarbonDictionaryWriter writer =
+          new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
+      for (String value : set[i]) {
+        writer.write(value);
+      }
+      writer.close();
+      writer.commit();
+      Dictionary dict = (Dictionary) dictCache.get(
+          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
+              columnIdentifier, dims.get(i).getDataType()));
+      CarbonDictionarySortInfoPreparator preparator =
+          new CarbonDictionarySortInfoPreparator();
+      List<String> newDistinctValues = new ArrayList<String>();
+      CarbonDictionarySortInfo dictionarySortInfo =
+          preparator.getDictionarySortInfo(newDistinctValues, dict, dims.get(i).getDataType());
+      CarbonDictionarySortIndexWriter carbonDictionaryWriter =
+          new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
+      try {
+        carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
+        carbonDictionaryWriter.writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());
+      } finally {
+        carbonDictionaryWriter.close();
+      }
+    }
+    reader.close();
+  }
+
+  /**
+   * Execute graph which will further load data
+   *
+   * @param loadModel
+   * @param storeLocation
+   * @throws Exception
+   */
+  public static void loadData(CarbonLoadModel loadModel, String storeLocation)
+      throws Exception {
+    if (new File(storeLocation).mkdirs()) {
+      LOG.warn("mkdir is failed");
+    }
+    String outPutLoc = storeLocation + "/etl";
+    String databaseName = loadModel.getDatabaseName();
+    String tableName = loadModel.getTableName();
+    String tempLocationKey = databaseName + '_' + tableName + "_1";
+    CarbonProperties.getInstance().addProperty(
+        tempLocationKey, storeLocation + "/" + databaseName + "/" + tableName);
+    CarbonProperties.getInstance().addProperty("store_output_location", outPutLoc);
+    CarbonProperties.getInstance().addProperty("send.signal.load", "false");
+    CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true");
+    CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1");
+    CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true");
+    CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true");
+    CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true");
+    CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false");
+    CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000");
+
+    String graphPath =
+        outPutLoc + File.separator + loadModel.getDatabaseName() + File.separator + tableName
+            + File.separator + 0 + File.separator + 1 + File.separator + tableName + ".ktr";
+    File path = new File(graphPath);
+    if (path.exists()) {
+      if (!path.delete()) {
+        LOG.warn("delete " + path + " failed");
+      }
+    }
+
+    BlockDetails blockDetails = new BlockDetails(new Path(loadModel.getFactFilePath()),
+        0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"});
+    Configuration configuration = new Configuration();
+    CSVInputFormat.setCommentCharacter(configuration, loadModel.getCommentChar());
+    CSVInputFormat.setCSVDelimiter(configuration, loadModel.getCsvDelimiter());
+    CSVInputFormat.setEscapeCharacter(configuration, loadModel.getEscapeChar());
+    CSVInputFormat.setHeaderExtractionEnabled(configuration, true);
+    CSVInputFormat.setQuoteCharacter(configuration, loadModel.getQuoteChar());
+    CSVInputFormat.setReadBufferSize(configuration,
+        CarbonProperties.getInstance().getProperty(
+            CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
+            CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
+    CSVInputFormat.setNumberOfColumns(
+        configuration, String.valueOf(loadModel.getCsvHeaderColumns().length));
+    CSVInputFormat.setMaxColumns(configuration, "10");
+
+    TaskAttemptContextImpl hadoopAttemptContext =
+        new TaskAttemptContextImpl(configuration, new TaskAttemptID("", 1, TaskType.MAP, 0, 0));
+    CSVInputFormat format = new CSVInputFormat();
+
+    RecordReader<NullWritable, StringArrayWritable> recordReader =
+        format.createRecordReader(blockDetails, hadoopAttemptContext);
+
+    CSVRecordReaderIterator readerIterator =
+        new CSVRecordReaderIterator(recordReader, blockDetails, hadoopAttemptContext);
+    new DataLoadExecutor().execute(loadModel,
+        new String[] {storeLocation + "/" + databaseName + "/" + tableName},
+        new CarbonIterator[]{readerIterator});
+
+    writeLoadMetadata(
+        loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(),
+        new ArrayList<LoadMetadataDetails>());
+  }
+
+  public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databaseName,
+      String tableName, List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
+    LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
+    loadMetadataDetails.setLoadEndTime(System.currentTimeMillis());
+    loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS);
+    loadMetadataDetails.setLoadName(String.valueOf(0));
+    loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
+    listOfLoadFolderDetails.add(loadMetadataDetails);
+
+    String dataLoadLocation = schema.getCarbonTable().getMetadataPath() + File.separator
+        + CarbonTablePath.TABLE_STATUS_FILE;
+
+    DataOutputStream dataOutputStream;
+    Gson gsonObjectToWrite = new Gson();
+    BufferedWriter brWriter = null;
+
+    AtomicFileOperations writeOperation =
+        new AtomicFileOperationsImpl(dataLoadLocation, FileFactory.getFileType(dataLoadLocation));
+
+    try {
+
+      dataOutputStream = writeOperation.openForWrite(FileWriteOperation.OVERWRITE);
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
+              Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      String metadataInstance = gsonObjectToWrite.toJson(listOfLoadFolderDetails.toArray());
+      brWriter.write(metadataInstance);
+    } finally {
+      try {
+        if (null != brWriter) {
+          brWriter.flush();
+        }
+      } catch (Exception e) {
+        throw e;
+
+      }
+      CarbonUtil.closeStreams(brWriter);
+
+    }
+    writeOperation.close();
+
+  }
+
+  public static String readCurrentTime() {
+    SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS);
+    String date = null;
+
+    date = sdf.format(new Date());
+
+    return date;
+  }
+
+  public static void main(String[] args) throws Exception {
+    StoreCreator.createCarbonStore();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonTypeUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonTypeUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonTypeUtil.java
deleted file mode 100644
index 395015e..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonTypeUtil.java
+++ /dev/null
@@ -1,101 +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.carbondata.hadoop.util;
-
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-
-import org.apache.spark.sql.types.DataTypes;
-import org.apache.spark.sql.types.DecimalType;
-import org.apache.spark.sql.types.StructField;
-
-public class CarbonTypeUtil {
-
-  public static org.apache.spark.sql.types.DataType convertCarbonToSparkDataType(
-      DataType carbonDataType) {
-    if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.STRING) {
-      return DataTypes.StringType;
-    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT) {
-      return DataTypes.ShortType;
-    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.INT) {
-      return DataTypes.IntegerType;
-    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.LONG) {
-      return DataTypes.LongType;
-    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE) {
-      return DataTypes.DoubleType;
-    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN) {
-      return DataTypes.BooleanType;
-    } else if (org.apache.carbondata.core.metadata.datatype.DataTypes.isDecimal(carbonDataType)) {
-      return DataTypes.createDecimalType();
-    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.TIMESTAMP) {
-      return DataTypes.TimestampType;
-    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DATE) {
-      return DataTypes.DateType;
-    } else {
-      return null;
-    }
-  }
-
-  public static StructField[] convertCarbonSchemaToSparkSchema(CarbonColumn[] carbonColumns) {
-    StructField[] fields = new StructField[carbonColumns.length];
-    for (int i = 0; i < carbonColumns.length; i++) {
-      CarbonColumn carbonColumn = carbonColumns[i];
-      if (carbonColumn.isDimension()) {
-        if (carbonColumn.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
-              .getDirectDictionaryGenerator(carbonColumn.getDataType());
-          fields[i] = new StructField(carbonColumn.getColName(),
-              CarbonTypeUtil.convertCarbonToSparkDataType(generator.getReturnType()), true, null);
-        } else if (!carbonColumn.hasEncoding(Encoding.DICTIONARY)) {
-          fields[i] = new StructField(carbonColumn.getColName(),
-              CarbonTypeUtil.convertCarbonToSparkDataType(carbonColumn.getDataType()), true, null);
-        } else if (carbonColumn.isComplex()) {
-          fields[i] = new StructField(carbonColumn.getColName(),
-              CarbonTypeUtil.convertCarbonToSparkDataType(carbonColumn.getDataType()), true, null);
-        } else {
-          fields[i] = new StructField(carbonColumn.getColName(), CarbonTypeUtil
-              .convertCarbonToSparkDataType(
-                  org.apache.carbondata.core.metadata.datatype.DataTypes.INT), true, null);
-        }
-      } else if (carbonColumn.isMeasure()) {
-        DataType dataType = carbonColumn.getDataType();
-        if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN
-            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT
-            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.INT
-            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.LONG) {
-          fields[i] = new StructField(carbonColumn.getColName(),
-              CarbonTypeUtil.convertCarbonToSparkDataType(dataType), true, null);
-        } else if (org.apache.carbondata.core.metadata.datatype.DataTypes.isDecimal(dataType)) {
-          CarbonMeasure measure = (CarbonMeasure) carbonColumn;
-          fields[i] = new StructField(carbonColumn.getColName(),
-              new DecimalType(measure.getPrecision(), measure.getScale()), true, null);
-        } else {
-          fields[i] = new StructField(carbonColumn.getColName(), CarbonTypeUtil
-              .convertCarbonToSparkDataType(
-                  org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE), true, null);
-        }
-      }
-    }
-    return fields;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
index 2f029ab..ea242d1 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableInputFormatTest.java
@@ -27,8 +27,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.UUID;
 
-import junit.framework.TestCase;
-
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -41,7 +39,7 @@ import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.hadoop.CarbonProjection;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
-import org.apache.carbondata.hadoop.test.util.StoreCreator;
+import org.apache.carbondata.hadoop.testutil.StoreCreator;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
@@ -52,9 +50,7 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
-import org.junit.After;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 
 public class CarbonTableInputFormatTest {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableOutputFormatTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableOutputFormatTest.java
index 653a49e..99f69c2 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableOutputFormatTest.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonTableOutputFormatTest.java
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
 import org.apache.carbondata.hadoop.internal.ObjectArrayWritable;
-import org.apache.carbondata.hadoop.test.util.StoreCreator;
+import org.apache.carbondata.hadoop.testutil.StoreCreator;
 import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
 import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormatTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormatTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormatTest.java
deleted file mode 100644
index 57f488f..0000000
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormatTest.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.hadoop.streaming;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.UUID;
-
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.statusmanager.FileFormat;
-import org.apache.carbondata.hadoop.CarbonInputSplit;
-import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
-import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
-
-import junit.framework.TestCase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class CarbonStreamInputFormatTest extends TestCase {
-
-  private TaskAttemptID taskAttemptId;
-  private TaskAttemptContext taskAttemptContext;
-  private Configuration hadoopConf;
-  private AbsoluteTableIdentifier identifier;
-  private String tablePath;
-
-
-  @Override protected void setUp() throws Exception {
-    tablePath = new File("target/stream_input").getCanonicalPath();
-    String dbName = "default";
-    String tableName = "stream_table_input";
-    identifier = AbsoluteTableIdentifier.from(
-        tablePath,
-        new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
-
-    JobID jobId = CarbonInputFormatUtil.getJobId(new Date(), 0);
-    TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
-    taskAttemptId = new TaskAttemptID(taskId, 0);
-
-    hadoopConf = new Configuration();
-    taskAttemptContext = new TaskAttemptContextImpl(hadoopConf, taskAttemptId);
-  }
-
-  private InputSplit buildInputSplit() throws IOException {
-    CarbonInputSplit carbonInputSplit = new CarbonInputSplit();
-    List<CarbonInputSplit> splitList = new ArrayList<>();
-    splitList.add(carbonInputSplit);
-    return new CarbonMultiBlockSplit(splitList, new String[] { "localhost" },
-        FileFormat.ROW_V1);
-  }
-
-  @Test public void testCreateRecordReader() {
-    try {
-      InputSplit inputSplit = buildInputSplit();
-      CarbonStreamInputFormat inputFormat = new CarbonStreamInputFormat();
-      RecordReader recordReader = inputFormat.createRecordReader(inputSplit, taskAttemptContext);
-      Assert.assertNotNull("Failed to create record reader", recordReader);
-    } catch (Exception e) {
-      e.printStackTrace();
-      Assert.assertTrue(e.getMessage(), false);
-    }
-  }
-
-  @Override protected void tearDown() throws Exception {
-    super.tearDown();
-    if (tablePath != null) {
-      FileFactory.deleteAllFilesOfDir(new File(tablePath));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormatTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormatTest.java
deleted file mode 100644
index e871c7e..0000000
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormatTest.java
+++ /dev/null
@@ -1,121 +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.carbondata.hadoop.streaming;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Date;
-import java.util.UUID;
-
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.hadoop.test.util.StoreCreator;
-import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-
-import junit.framework.TestCase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class CarbonStreamOutputFormatTest extends TestCase {
-
-  private Configuration hadoopConf;
-  private TaskAttemptID taskAttemptId;
-  private CarbonLoadModel carbonLoadModel;
-  private String tablePath;
-
-  @Override protected void setUp() throws Exception {
-    super.setUp();
-    JobID jobId = CarbonInputFormatUtil.getJobId(new Date(), 0);
-    TaskID taskId = new TaskID(jobId, TaskType.MAP, 0);
-    taskAttemptId = new TaskAttemptID(taskId, 0);
-
-    hadoopConf = new Configuration();
-    hadoopConf.set("mapred.job.id", jobId.toString());
-    hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID().toString());
-    hadoopConf.set("mapred.task.id", taskAttemptId.toString());
-    hadoopConf.setBoolean("mapred.task.is.map", true);
-    hadoopConf.setInt("mapred.task.partition", 0);
-
-    tablePath = new File("target/stream_output").getCanonicalPath();
-    String dbName = "default";
-    String tableName = "stream_table_output";
-    AbsoluteTableIdentifier identifier =
-        AbsoluteTableIdentifier.from(
-            tablePath,
-            new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
-
-    CarbonTable table = StoreCreator.createTable(identifier);
-
-    String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
-    carbonLoadModel = StoreCreator.buildCarbonLoadModel(table, factFilePath, identifier);
-  }
-
-  @Test public void testSetCarbonLoadModel() {
-    try {
-      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
-    } catch (IOException e) {
-      Assert.assertTrue("Failed to config CarbonLoadModel for CarbonStreamOutputFromat", false);
-    }
-  }
-
-  @Test public void testGetCarbonLoadModel() {
-    try {
-      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
-      CarbonLoadModel model = CarbonStreamOutputFormat.getCarbonLoadModel(hadoopConf);
-
-      Assert.assertNotNull("Failed to get CarbonLoadModel", model);
-      Assert.assertTrue("CarbonLoadModel should be same with previous",
-          carbonLoadModel.getFactTimeStamp() == model.getFactTimeStamp());
-
-    } catch (IOException e) {
-      Assert.assertTrue("Failed to get CarbonLoadModel for CarbonStreamOutputFromat", false);
-    }
-  }
-
-  @Test public void testGetRecordWriter() {
-    CarbonStreamOutputFormat outputFormat = new CarbonStreamOutputFormat();
-    try {
-      CarbonStreamOutputFormat.setCarbonLoadModel(hadoopConf, carbonLoadModel);
-      TaskAttemptContext taskAttemptContext =
-          new TaskAttemptContextImpl(hadoopConf, taskAttemptId);
-      RecordWriter recordWriter = outputFormat.getRecordWriter(taskAttemptContext);
-      Assert.assertNotNull("Failed to get CarbonStreamRecordWriter", recordWriter);
-    } catch (Exception e) {
-      e.printStackTrace();
-      Assert.assertTrue(e.getMessage(), false);
-    }
-  }
-
-  @Override protected void tearDown() throws Exception {
-    super.tearDown();
-    if (tablePath != null) {
-      FileFactory.deleteAllFilesOfDir(new File(tablePath));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
deleted file mode 100644
index 8e8916d..0000000
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ /dev/null
@@ -1,492 +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.carbondata.hadoop.test.util;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.nio.charset.Charset;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.DataMapStoreManager;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
-import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
-import org.apache.carbondata.core.fileoperations.FileWriteOperation;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.converter.SchemaConverter;
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
-import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.TableInfo;
-import org.apache.carbondata.core.metadata.schema.table.TableSchema;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.core.statusmanager.SegmentStatus;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
-import org.apache.carbondata.core.writer.ThriftWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfoPreparator;
-import org.apache.carbondata.processing.loading.DataLoadExecutor;
-import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
-import org.apache.carbondata.processing.loading.csvinput.BlockDetails;
-import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
-import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator;
-import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable;
-import org.apache.carbondata.processing.loading.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-import org.apache.carbondata.processing.util.TableOptionConstant;
-
-import com.google.gson.Gson;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-
-/**
- * This class will create store file based on provided schema
- *
- */
-public class StoreCreator {
-
-  private static AbsoluteTableIdentifier absoluteTableIdentifier;
-  private static String storePath = null;
-
-  static {
-    try {
-      storePath = new File("target/store").getCanonicalPath();
-      String dbName = "testdb";
-      String tableName = "testtable";
-      absoluteTableIdentifier =
-          AbsoluteTableIdentifier.from(
-              storePath +"/testdb/testtable",
-              new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
-    } catch (IOException ex) {
-
-    }
-  }
-
-  public static AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  public static CarbonLoadModel buildCarbonLoadModel(CarbonTable table, String factFilePath,
-      AbsoluteTableIdentifier absoluteTableIdentifier) {
-    CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
-    CarbonLoadModel loadModel = new CarbonLoadModel();
-    loadModel.setCarbonDataLoadSchema(schema);
-    loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
-    loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-    loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-    loadModel.setFactFilePath(factFilePath);
-    loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
-    loadModel.setTablePath(absoluteTableIdentifier.getTablePath());
-    loadModel.setDateFormat(null);
-    loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
-        CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS));
-    loadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty(
-        CarbonCommonConstants.CARBON_DATE_FORMAT,
-        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
-    loadModel
-        .setSerializationNullFormat(
-            TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N");
-    loadModel
-        .setBadRecordsLoggerEnable(
-            TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false");
-    loadModel
-        .setBadRecordsAction(
-            TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "FORCE");
-    loadModel
-        .setIsEmptyDataBadRecord(
-            DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + "false");
-    loadModel.setCsvHeader("ID,date,country,name,phonetype,serialname,salary");
-    loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(","));
-    loadModel.setTaskNo("0");
-    loadModel.setSegmentId("0");
-    loadModel.setFactTimeStamp(System.currentTimeMillis());
-    loadModel.setMaxColumns("10");
-    return loadModel;
-  }
-
-  /**
-   * Create store without any restructure
-   */
-  public static void createCarbonStore() throws Exception {
-    CarbonLoadModel loadModel = createTableAndLoadModel();
-    loadData(loadModel, storePath);
-  }
-
-  /**
-   * Method to clear the data maps
-   */
-  public static void clearDataMaps() {
-    DataMapStoreManager.getInstance().clearDataMaps(absoluteTableIdentifier);
-  }
-
-  public static CarbonLoadModel createTableAndLoadModel() throws Exception {
-    String factFilePath =
-        new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
-    File storeDir = new File(storePath);
-    CarbonUtil.deleteFoldersAndFiles(storeDir);
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS,
-        storePath);
-
-    CarbonTable table = createTable(absoluteTableIdentifier);
-    writeDictionary(factFilePath, table);
-    return buildCarbonLoadModel(table, factFilePath, absoluteTableIdentifier);
-  }
-
-  public static CarbonTable createTable(
-      AbsoluteTableIdentifier identifier) throws IOException {
-    TableInfo tableInfo = new TableInfo();
-    tableInfo.setDatabaseName(identifier.getCarbonTableIdentifier().getDatabaseName());
-    TableSchema tableSchema = new TableSchema();
-    tableSchema.setTableName(identifier.getCarbonTableIdentifier().getTableName());
-    List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
-    ArrayList<Encoding> encodings = new ArrayList<>();
-    encodings.add(Encoding.DICTIONARY);
-    ColumnSchema id = new ColumnSchema();
-    id.setColumnName("ID");
-    id.setColumnar(true);
-    id.setDataType(DataTypes.INT);
-    id.setEncodingList(encodings);
-    id.setColumnUniqueId(UUID.randomUUID().toString());
-    id.setColumnReferenceId(id.getColumnUniqueId());
-    id.setDimensionColumn(true);
-    id.setColumnGroup(1);
-    columnSchemas.add(id);
-
-    ColumnSchema date = new ColumnSchema();
-    date.setColumnName("date");
-    date.setColumnar(true);
-    date.setDataType(DataTypes.STRING);
-    date.setEncodingList(encodings);
-    date.setColumnUniqueId(UUID.randomUUID().toString());
-    date.setDimensionColumn(true);
-    date.setColumnGroup(2);
-    date.setSortColumn(true);
-    date.setColumnReferenceId(id.getColumnUniqueId());
-    columnSchemas.add(date);
-
-    ColumnSchema country = new ColumnSchema();
-    country.setColumnName("country");
-    country.setColumnar(true);
-    country.setDataType(DataTypes.STRING);
-    country.setEncodingList(encodings);
-    country.setColumnUniqueId(UUID.randomUUID().toString());
-    country.setDimensionColumn(true);
-    country.setColumnGroup(3);
-    country.setSortColumn(true);
-    country.setColumnReferenceId(id.getColumnUniqueId());
-    columnSchemas.add(country);
-
-    ColumnSchema name = new ColumnSchema();
-    name.setColumnName("name");
-    name.setColumnar(true);
-    name.setDataType(DataTypes.STRING);
-    name.setEncodingList(encodings);
-    name.setColumnUniqueId(UUID.randomUUID().toString());
-    name.setDimensionColumn(true);
-    name.setColumnGroup(4);
-    name.setSortColumn(true);
-    name.setColumnReferenceId(id.getColumnUniqueId());
-    columnSchemas.add(name);
-
-    ColumnSchema phonetype = new ColumnSchema();
-    phonetype.setColumnName("phonetype");
-    phonetype.setColumnar(true);
-    phonetype.setDataType(DataTypes.STRING);
-    phonetype.setEncodingList(encodings);
-    phonetype.setColumnUniqueId(UUID.randomUUID().toString());
-    phonetype.setDimensionColumn(true);
-    phonetype.setColumnGroup(5);
-    phonetype.setSortColumn(true);
-    phonetype.setColumnReferenceId(id.getColumnUniqueId());
-    columnSchemas.add(phonetype);
-
-    ColumnSchema serialname = new ColumnSchema();
-    serialname.setColumnName("serialname");
-    serialname.setColumnar(true);
-    serialname.setDataType(DataTypes.STRING);
-    serialname.setEncodingList(encodings);
-    serialname.setColumnUniqueId(UUID.randomUUID().toString());
-    serialname.setDimensionColumn(true);
-    serialname.setColumnGroup(6);
-    serialname.setSortColumn(true);
-    serialname.setColumnReferenceId(id.getColumnUniqueId());
-    columnSchemas.add(serialname);
-
-    ColumnSchema salary = new ColumnSchema();
-    salary.setColumnName("salary");
-    salary.setColumnar(true);
-    salary.setDataType(DataTypes.INT);
-    salary.setEncodingList(new ArrayList<Encoding>());
-    salary.setColumnUniqueId(UUID.randomUUID().toString());
-    salary.setDimensionColumn(false);
-    salary.setColumnReferenceId(id.getColumnUniqueId());
-    salary.setColumnGroup(7);
-    columnSchemas.add(salary);
-
-    tableSchema.setListOfColumns(columnSchemas);
-    SchemaEvolution schemaEvol = new SchemaEvolution();
-    schemaEvol.setSchemaEvolutionEntryList(new ArrayList<SchemaEvolutionEntry>());
-    tableSchema.setSchemaEvalution(schemaEvol);
-    tableSchema.setTableId(UUID.randomUUID().toString());
-    tableInfo.setTableUniqueName(
-        identifier.getCarbonTableIdentifier().getTableUniqueName()
-    );
-    tableInfo.setLastUpdatedTime(System.currentTimeMillis());
-    tableInfo.setFactTable(tableSchema);
-    tableInfo.setTablePath(identifier.getTablePath());
-    String schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath());
-    String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
-    CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
-
-    SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
-    org.apache.carbondata.format.TableInfo thriftTableInfo =
-        schemaConverter.fromWrapperToExternalTableInfo(
-            tableInfo,
-            tableInfo.getDatabaseName(),
-            tableInfo.getFactTable().getTableName());
-    org.apache.carbondata.format.SchemaEvolutionEntry schemaEvolutionEntry =
-        new org.apache.carbondata.format.SchemaEvolutionEntry(tableInfo.getLastUpdatedTime());
-    thriftTableInfo.getFact_table().getSchema_evolution().getSchema_evolution_history()
-        .add(schemaEvolutionEntry);
-
-    FileFactory.FileType fileType = FileFactory.getFileType(schemaMetadataPath);
-    if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
-      FileFactory.mkdirs(schemaMetadataPath, fileType);
-    }
-
-    ThriftWriter thriftWriter = new ThriftWriter(schemaFilePath, false);
-    thriftWriter.open();
-    thriftWriter.write(thriftTableInfo);
-    thriftWriter.close();
-    return CarbonMetadata.getInstance().getCarbonTable(tableInfo.getTableUniqueName());
-  }
-
-  private static void writeDictionary(String factFilePath, CarbonTable table) throws Exception {
-    BufferedReader reader = new BufferedReader(new FileReader(factFilePath));
-    String header = reader.readLine();
-    String[] split = header.split(",");
-    List<CarbonColumn> allCols = new ArrayList<CarbonColumn>();
-    List<CarbonDimension> dims = table.getDimensionByTableName(table.getTableName());
-    allCols.addAll(dims);
-    List<CarbonMeasure> msrs = table.getMeasureByTableName(table.getTableName());
-    allCols.addAll(msrs);
-    Set<String>[] set = new HashSet[dims.size()];
-    for (int i = 0; i < set.length; i++) {
-      set[i] = new HashSet<String>();
-    }
-    String line = reader.readLine();
-    while (line != null) {
-      String[] data = line.split(",");
-      for (int i = 0; i < set.length; i++) {
-        set[i].add(data[i]);
-      }
-      line = reader.readLine();
-    }
-
-    Cache dictCache = CacheProvider.getInstance()
-        .createCache(CacheType.REVERSE_DICTIONARY);
-    for (int i = 0; i < set.length; i++) {
-      ColumnIdentifier columnIdentifier = new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-          new DictionaryColumnUniqueIdentifier(table.getAbsoluteTableIdentifier(), columnIdentifier,
-              columnIdentifier.getDataType());
-      CarbonDictionaryWriter writer =
-          new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-      for (String value : set[i]) {
-        writer.write(value);
-      }
-      writer.close();
-      writer.commit();
-      Dictionary dict = (Dictionary) dictCache.get(
-          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
-        		  columnIdentifier, dims.get(i).getDataType()));
-      CarbonDictionarySortInfoPreparator preparator =
-          new CarbonDictionarySortInfoPreparator();
-      List<String> newDistinctValues = new ArrayList<String>();
-      CarbonDictionarySortInfo dictionarySortInfo =
-          preparator.getDictionarySortInfo(newDistinctValues, dict, dims.get(i).getDataType());
-      CarbonDictionarySortIndexWriter carbonDictionaryWriter =
-          new CarbonDictionarySortIndexWriterImpl(dictionaryColumnUniqueIdentifier);
-      try {
-        carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
-        carbonDictionaryWriter.writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());
-      } finally {
-        carbonDictionaryWriter.close();
-      }
-    }
-    reader.close();
-  }
-
-  /**
-   * Execute graph which will further load data
-   *
-   * @param loadModel
-   * @param storeLocation
-   * @throws Exception
-   */
-  public static void loadData(CarbonLoadModel loadModel, String storeLocation)
-      throws Exception {
-    new File(storeLocation).mkdirs();
-    String outPutLoc = storeLocation + "/etl";
-    String databaseName = loadModel.getDatabaseName();
-    String tableName = loadModel.getTableName();
-    String tempLocationKey = databaseName + '_' + tableName + "_1";
-    CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation + "/" + databaseName + "/" + tableName);
-    CarbonProperties.getInstance().addProperty("store_output_location", outPutLoc);
-    CarbonProperties.getInstance().addProperty("send.signal.load", "false");
-    CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true");
-    CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1");
-    CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true");
-    CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true");
-    CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true");
-    CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false");
-    CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000");
-
-    String graphPath =
-        outPutLoc + File.separator + loadModel.getDatabaseName() + File.separator + tableName
-            + File.separator + 0 + File.separator + 1 + File.separator + tableName + ".ktr";
-    File path = new File(graphPath);
-    if (path.exists()) {
-      path.delete();
-    }
-
-    BlockDetails blockDetails = new BlockDetails(new Path(loadModel.getFactFilePath()),
-        0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"});
-    Configuration configuration = new Configuration();
-    CSVInputFormat.setCommentCharacter(configuration, loadModel.getCommentChar());
-    CSVInputFormat.setCSVDelimiter(configuration, loadModel.getCsvDelimiter());
-    CSVInputFormat.setEscapeCharacter(configuration, loadModel.getEscapeChar());
-    CSVInputFormat.setHeaderExtractionEnabled(configuration, true);
-    CSVInputFormat.setQuoteCharacter(configuration, loadModel.getQuoteChar());
-    CSVInputFormat.setReadBufferSize(configuration, CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
-            CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
-    CSVInputFormat.setNumberOfColumns(configuration, String.valueOf(loadModel.getCsvHeaderColumns().length));
-    CSVInputFormat.setMaxColumns(configuration, "10");
-
-    TaskAttemptContextImpl hadoopAttemptContext = new TaskAttemptContextImpl(configuration, new TaskAttemptID("", 1, TaskType.MAP, 0, 0));
-    CSVInputFormat format = new CSVInputFormat();
-
-    RecordReader<NullWritable, StringArrayWritable> recordReader =
-        format.createRecordReader(blockDetails, hadoopAttemptContext);
-
-    CSVRecordReaderIterator readerIterator = new CSVRecordReaderIterator(recordReader, blockDetails, hadoopAttemptContext);
-    new DataLoadExecutor().execute(loadModel,
-        new String[] {storeLocation + "/" + databaseName + "/" + tableName},
-        new CarbonIterator[]{readerIterator});
-
-    writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(),
-        new ArrayList<LoadMetadataDetails>());
-  }
-
-  public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databaseName,
-      String tableName, List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
-    LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
-    loadMetadataDetails.setLoadEndTime(System.currentTimeMillis());
-    loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS);
-    loadMetadataDetails.setLoadName(String.valueOf(0));
-    loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
-    listOfLoadFolderDetails.add(loadMetadataDetails);
-
-    String dataLoadLocation = schema.getCarbonTable().getMetadataPath() + File.separator
-        + CarbonTablePath.TABLE_STATUS_FILE;
-
-    DataOutputStream dataOutputStream;
-    Gson gsonObjectToWrite = new Gson();
-    BufferedWriter brWriter = null;
-
-    AtomicFileOperations writeOperation =
-        new AtomicFileOperationsImpl(dataLoadLocation, FileFactory.getFileType(dataLoadLocation));
-
-    try {
-
-      dataOutputStream = writeOperation.openForWrite(FileWriteOperation.OVERWRITE);
-      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
-              Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-
-      String metadataInstance = gsonObjectToWrite.toJson(listOfLoadFolderDetails.toArray());
-      brWriter.write(metadataInstance);
-    } finally {
-      try {
-        if (null != brWriter) {
-          brWriter.flush();
-        }
-      } catch (Exception e) {
-        throw e;
-
-      }
-      CarbonUtil.closeStreams(brWriter);
-
-    }
-    writeOperation.close();
-
-  }
-
-  public static String readCurrentTime() {
-    SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_MILLIS);
-    String date = null;
-
-    date = sdf.format(new Date());
-
-    return date;
-  }
-
-  public static void main(String[] args) throws Exception {
-    StoreCreator.createCarbonStore();
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common/pom.xml b/integration/spark-common/pom.xml
index 16f327d..f011a75 100644
--- a/integration/spark-common/pom.xml
+++ b/integration/spark-common/pom.xml
@@ -36,7 +36,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
-      <artifactId>carbondata-hadoop</artifactId>
+      <artifactId>carbondata-streaming</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/SparkDataTypeConverterImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/SparkDataTypeConverterImpl.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/SparkDataTypeConverterImpl.java
index 6e9e0a6..f6dc65b 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/SparkDataTypeConverterImpl.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/SparkDataTypeConverterImpl.java
@@ -20,10 +20,19 @@ package org.apache.carbondata.spark.util;
 import java.io.Serializable;
 import java.math.BigDecimal;
 
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.util.DataTypeConverter;
 
 import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
 import org.apache.spark.sql.catalyst.util.GenericArrayData;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.DecimalType;
+import org.apache.spark.sql.types.StructField;
 import org.apache.spark.unsafe.types.UTF8String;
 
 /**
@@ -90,4 +99,76 @@ public final class SparkDataTypeConverterImpl implements DataTypeConverter, Seri
   public Object wrapWithGenericRow(Object[] fields) {
     return new GenericInternalRow(fields);
   }
+
+  private static org.apache.spark.sql.types.DataType convertCarbonToSparkDataType(
+      DataType carbonDataType) {
+    if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.STRING) {
+      return DataTypes.StringType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT) {
+      return DataTypes.ShortType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.INT) {
+      return DataTypes.IntegerType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.LONG) {
+      return DataTypes.LongType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE) {
+      return DataTypes.DoubleType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN) {
+      return DataTypes.BooleanType;
+    } else if (org.apache.carbondata.core.metadata.datatype.DataTypes.isDecimal(carbonDataType)) {
+      return DataTypes.createDecimalType();
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.TIMESTAMP) {
+      return DataTypes.TimestampType;
+    } else if (carbonDataType == org.apache.carbondata.core.metadata.datatype.DataTypes.DATE) {
+      return DataTypes.DateType;
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * convert from CarbonColumn array to Spark's StructField array
+   */
+  @Override
+  public Object[] convertCarbonSchemaToSparkSchema(CarbonColumn[] carbonColumns) {
+    StructField[] fields = new StructField[carbonColumns.length];
+    for (int i = 0; i < carbonColumns.length; i++) {
+      CarbonColumn carbonColumn = carbonColumns[i];
+      if (carbonColumn.isDimension()) {
+        if (carbonColumn.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
+              .getDirectDictionaryGenerator(carbonColumn.getDataType());
+          fields[i] = new StructField(carbonColumn.getColName(),
+              convertCarbonToSparkDataType(generator.getReturnType()), true, null);
+        } else if (!carbonColumn.hasEncoding(Encoding.DICTIONARY)) {
+          fields[i] = new StructField(carbonColumn.getColName(),
+              convertCarbonToSparkDataType(carbonColumn.getDataType()), true, null);
+        } else if (carbonColumn.isComplex()) {
+          fields[i] = new StructField(carbonColumn.getColName(),
+              convertCarbonToSparkDataType(carbonColumn.getDataType()), true, null);
+        } else {
+          fields[i] = new StructField(carbonColumn.getColName(),
+              convertCarbonToSparkDataType(
+                  org.apache.carbondata.core.metadata.datatype.DataTypes.INT), true, null);
+        }
+      } else if (carbonColumn.isMeasure()) {
+        DataType dataType = carbonColumn.getDataType();
+        if (dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.BOOLEAN
+            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.SHORT
+            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.INT
+            || dataType == org.apache.carbondata.core.metadata.datatype.DataTypes.LONG) {
+          fields[i] = new StructField(carbonColumn.getColName(),
+              convertCarbonToSparkDataType(dataType), true, null);
+        } else if (org.apache.carbondata.core.metadata.datatype.DataTypes.isDecimal(dataType)) {
+          CarbonMeasure measure = (CarbonMeasure) carbonColumn;
+          fields[i] = new StructField(carbonColumn.getColName(),
+              new DecimalType(measure.getPrecision(), measure.getScale()), true, null);
+        } else {
+          fields[i] = new StructField(carbonColumn.getColName(),
+              convertCarbonToSparkDataType(
+                  org.apache.carbondata.core.metadata.datatype.DataTypes.DOUBLE), true, null);
+        }
+      }
+    }
+    return fields;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 2be0efc..7e549a6 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -53,10 +53,11 @@ import org.apache.carbondata.core.statusmanager.FileFormat
 import org.apache.carbondata.core.util._
 import org.apache.carbondata.hadoop._
 import org.apache.carbondata.hadoop.api.{CarbonFileInputFormat, CarbonInputFormat, CarbonTableInputFormat}
-import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
+import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.InitInputMetrics
 import org.apache.carbondata.spark.util.{SparkDataTypeConverterImpl, Util}
+import org.apache.carbondata.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
 
 /**
  * This RDD is used to perform query on CarbonData file. Before sending tasks to scan


[5/5] carbondata git commit: [CARBONDATA-2165]Remove spark in carbon-hadoop module

Posted by ja...@apache.org.
[CARBONDATA-2165]Remove spark in carbon-hadoop module

1. Streaming relation RecordReader is moved to carbon-streaming module.
2. RDD related class is moved to carbon-spark2 module

This closes #2074


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

Branch: refs/heads/master
Commit: c723947a79332c66175f5a33cf57f08fe70fe1a9
Parents: 2e1ddb5
Author: Jacky Li <ja...@qq.com>
Authored: Sat Mar 17 18:13:08 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Wed Mar 28 11:19:23 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/util/DataTypeConverter.java |   3 +
 .../core/util/DataTypeConverterImpl.java        |   5 +
 hadoop/CARBON_HADOOPLogResource.properties      |  18 -
 hadoop/pom.xml                                  |   7 -
 .../readsupport/impl/RawDataReadSupport.java    |  42 -
 .../streaming/CarbonStreamInputFormat.java      | 115 ---
 .../streaming/CarbonStreamOutputFormat.java     |  87 ---
 .../streaming/CarbonStreamRecordReader.java     | 759 ------------------
 .../streaming/CarbonStreamRecordWriter.java     | 325 --------
 .../hadoop/streaming/StreamBlockletReader.java  | 259 -------
 .../hadoop/streaming/StreamBlockletWriter.java  | 152 ----
 .../hadoop/testutil/StoreCreator.java           | 495 ++++++++++++
 .../carbondata/hadoop/util/CarbonTypeUtil.java  | 101 ---
 .../hadoop/ft/CarbonTableInputFormatTest.java   |   6 +-
 .../hadoop/ft/CarbonTableOutputFormatTest.java  |   2 +-
 .../streaming/CarbonStreamInputFormatTest.java  |  99 ---
 .../streaming/CarbonStreamOutputFormatTest.java | 121 ---
 .../hadoop/test/util/StoreCreator.java          | 492 ------------
 integration/spark-common/pom.xml                |   2 +-
 .../spark/util/SparkDataTypeConverterImpl.java  |  81 ++
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   3 +-
 .../carbondata/spark/rdd/StreamHandoffRDD.scala | 435 +++++++++++
 .../carbondata/spark/util/CarbonScalaUtil.scala |  52 +-
 .../CarbonSparkStreamingListener.scala          |  30 +
 .../streaming/CarbonStreamSparkStreaming.scala  | 184 +++++
 .../CarbonStreamingQueryListener.scala          |  77 ++
 .../streaming/StreamSinkFactory.scala           | 236 ++++++
 .../streaming/CarbonAppendableStreamSink.scala  | 362 +++++++++
 .../spark/sql/test/TestQueryExecutor.scala      |   4 +-
 integration/spark2/pom.xml                      |   2 +-
 .../org/apache/spark/sql/CarbonSession.scala    |   2 +-
 .../CarbonAlterTableCompactionCommand.scala     |   3 +-
 streaming/pom.xml                               |   9 +-
 .../streaming/CarbonStreamInputFormat.java      | 115 +++
 .../streaming/CarbonStreamOutputFormat.java     |  87 +++
 .../streaming/CarbonStreamRecordReader.java     | 761 +++++++++++++++++++
 .../streaming/CarbonStreamRecordWriter.java     | 325 ++++++++
 .../streaming/StreamBlockletReader.java         | 259 +++++++
 .../streaming/StreamBlockletWriter.java         | 152 ++++
 .../streaming/segment/StreamSegment.java        |   2 +-
 .../CarbonSparkStreamingListener.scala          |  31 -
 .../streaming/CarbonStreamSparkStreaming.scala  | 187 -----
 .../carbondata/streaming/StreamHandoffRDD.scala | 436 -----------
 .../streaming/StreamSinkFactory.scala           | 236 ------
 .../streaming/parser/FieldConverter.scala       |  95 +++
 .../streaming/parser/RowStreamParserImp.scala   |   7 +-
 .../streaming/CarbonAppendableStreamSink.scala  | 362 ---------
 .../CarbonStreamingQueryListener.scala          |  77 --
 .../streaming/CarbonStreamInputFormatTest.java  |  99 +++
 .../streaming/CarbonStreamOutputFormatTest.java | 121 +++
 50 files changed, 3948 insertions(+), 3974 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverter.java b/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverter.java
index 7c63860..474493a 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverter.java
@@ -17,6 +17,8 @@
 
 package org.apache.carbondata.core.util;
 
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+
 public interface DataTypeConverter {
 
   Object convertFromStringToDecimal(Object data);
@@ -31,4 +33,5 @@ public interface DataTypeConverter {
   Object wrapWithGenericArrayData(Object data);
   Object wrapWithGenericRow(Object[] fields);
 
+  Object[] convertCarbonSchemaToSparkSchema(CarbonColumn[] carbonColumns);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverterImpl.java
index ea5740d..a4f571e 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataTypeConverterImpl.java
@@ -21,6 +21,7 @@ import java.io.Serializable;
 import java.math.BigDecimal;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 
 public class DataTypeConverterImpl implements DataTypeConverter, Serializable {
 
@@ -91,4 +92,8 @@ public class DataTypeConverterImpl implements DataTypeConverter, Serializable {
     return fields;
   }
 
+  @Override
+  public Object[] convertCarbonSchemaToSparkSchema(CarbonColumn[] carbonColumns) {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/CARBON_HADOOPLogResource.properties
----------------------------------------------------------------------
diff --git a/hadoop/CARBON_HADOOPLogResource.properties b/hadoop/CARBON_HADOOPLogResource.properties
deleted file mode 100644
index 135a578..0000000
--- a/hadoop/CARBON_HADOOPLogResource.properties
+++ /dev/null
@@ -1,18 +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.
-#
-carbon.hadoop = {0}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop/pom.xml b/hadoop/pom.xml
index 916b9db..41e2822 100644
--- a/hadoop/pom.xml
+++ b/hadoop/pom.xml
@@ -40,10 +40,6 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_${scala.binary.version}</artifactId>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
@@ -55,9 +51,6 @@
     <resources>
       <resource>
         <directory>.</directory>
-        <includes>
-          <include>CARBON_HADOOPLogResource.properties</include>
-        </includes>
       </resource>
     </resources>
     <plugins>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java
deleted file mode 100644
index b2cd450..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/RawDataReadSupport.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.hadoop.readsupport.impl;
-
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
-
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
-
-public class RawDataReadSupport implements CarbonReadSupport<InternalRow> {
-
-  @Override
-  public void initialize(CarbonColumn[] carbonColumns, CarbonTable carbonTable) { }
-
-  /**
-   * return column data as InternalRow
-   *
-   * @param data column data
-   */
-  @Override
-  public InternalRow readRow(Object[] data) {
-    return new GenericInternalRow(data);
-  }
-
-  @Override public void close() { }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.java
deleted file mode 100644
index a6e9563..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamInputFormat.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.carbondata.hadoop.streaming;
-
-import java.io.IOException;
-
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.scan.complextypes.ArrayQueryType;
-import org.apache.carbondata.core.scan.complextypes.PrimitiveQueryType;
-import org.apache.carbondata.core.scan.complextypes.StructQueryType;
-import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-
-/**
- * Stream input format
- */
-public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
-
-  public static final String READ_BUFFER_SIZE = "carbon.stream.read.buffer.size";
-  public static final String READ_BUFFER_SIZE_DEFAULT = "65536";
-
-  @Override public RecordReader<Void, Object> createRecordReader(InputSplit split,
-      TaskAttemptContext context) throws IOException, InterruptedException {
-    return new CarbonStreamRecordReader();
-  }
-
-  public static GenericQueryType[] getComplexDimensions(CarbonTable carbontable,
-      CarbonColumn[] carbonColumns, Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache)
-      throws IOException {
-    GenericQueryType[] queryTypes = new GenericQueryType[carbonColumns.length];
-    for (int i = 0; i < carbonColumns.length; i++) {
-      if (carbonColumns[i].isComplex()) {
-        if (DataTypes.isArrayType(carbonColumns[i].getDataType())) {
-          queryTypes[i] = new ArrayQueryType(carbonColumns[i].getColName(),
-              carbonColumns[i].getColName(), i);
-        } else if (DataTypes.isStructType(carbonColumns[i].getDataType())) {
-          queryTypes[i] = new StructQueryType(carbonColumns[i].getColName(),
-              carbonColumns[i].getColName(), i);
-        } else {
-          throw new UnsupportedOperationException(
-              carbonColumns[i].getDataType().getName() + " is not supported");
-        }
-
-        fillChildren(carbontable, queryTypes[i], (CarbonDimension) carbonColumns[i], i, cache);
-      }
-    }
-
-    return queryTypes;
-  }
-
-  private static void fillChildren(CarbonTable carbontable, GenericQueryType parentQueryType,
-      CarbonDimension dimension, int parentBlockIndex,
-      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache) throws IOException {
-    for (int i = 0; i < dimension.getNumberOfChild(); i++) {
-      CarbonDimension child = dimension.getListOfChildDimensions().get(i);
-      DataType dataType = child.getDataType();
-      GenericQueryType queryType = null;
-      if (DataTypes.isArrayType(dataType)) {
-        queryType =
-            new ArrayQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
-
-      } else if (DataTypes.isStructType(dataType)) {
-        queryType =
-            new StructQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
-        parentQueryType.addChildren(queryType);
-      } else {
-        boolean isDirectDictionary =
-            CarbonUtil.hasEncoding(child.getEncoder(), Encoding.DIRECT_DICTIONARY);
-        String dictionaryPath = carbontable.getTableInfo().getFactTable().getTableProperties()
-            .get(CarbonCommonConstants.DICTIONARY_PATH);
-        DictionaryColumnUniqueIdentifier dictionarIdentifier =
-            new DictionaryColumnUniqueIdentifier(carbontable.getAbsoluteTableIdentifier(),
-                child.getColumnIdentifier(), child.getDataType(), dictionaryPath);
-
-        queryType =
-            new PrimitiveQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex,
-                child.getDataType(), 4, cache.get(dictionarIdentifier),
-                isDirectDictionary);
-      }
-      parentQueryType.addChildren(queryType);
-      if (child.getNumberOfChild() > 0) {
-        fillChildren(carbontable, queryType, child, parentBlockIndex, cache);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
deleted file mode 100644
index 2599fa7..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamOutputFormat.java
+++ /dev/null
@@ -1,87 +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.carbondata.hadoop.streaming;
-
-import java.io.IOException;
-import java.nio.charset.Charset;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-
-/**
- * Stream output format
- */
-public class CarbonStreamOutputFormat extends FileOutputFormat<Void, Object> {
-
-  static final byte[] CARBON_SYNC_MARKER =
-      "@carbondata_sync".getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-
-  public static final String CARBON_ENCODER_ROW_BUFFER_SIZE = "carbon.stream.row.buffer.size";
-
-  public static final int CARBON_ENCODER_ROW_BUFFER_SIZE_DEFAULT = 1024;
-
-  public static final String CARBON_STREAM_BLOCKLET_ROW_NUMS = "carbon.stream.blocklet.row.nums";
-
-  public static final int CARBON_STREAM_BLOCKLET_ROW_NUMS_DEFAULT = 32000;
-
-  public static final String CARBON_STREAM_CACHE_SIZE = "carbon.stream.cache.size";
-
-  public static final int CARBON_STREAM_CACHE_SIZE_DEFAULT = 32 * 1024 * 1024;
-
-  private static final String LOAD_Model = "mapreduce.output.carbon.load.model";
-
-  private static final String SEGMENT_ID = "carbon.segment.id";
-
-  @Override public RecordWriter<Void, Object> getRecordWriter(TaskAttemptContext job)
-      throws IOException, InterruptedException {
-    return new CarbonStreamRecordWriter(job);
-  }
-
-  public static void setCarbonLoadModel(Configuration hadoopConf, CarbonLoadModel carbonLoadModel)
-      throws IOException {
-    if (carbonLoadModel != null) {
-      hadoopConf.set(LOAD_Model, ObjectSerializationUtil.convertObjectToString(carbonLoadModel));
-    }
-  }
-
-  public static CarbonLoadModel getCarbonLoadModel(Configuration hadoopConf) throws IOException {
-    String value = hadoopConf.get(LOAD_Model);
-    if (value == null) {
-      return null;
-    } else {
-      return (CarbonLoadModel) ObjectSerializationUtil.convertStringToObject(value);
-    }
-  }
-
-  public static void setSegmentId(Configuration hadoopConf, String segmentId) throws IOException {
-    if (segmentId != null) {
-      hadoopConf.set(SEGMENT_ID, segmentId);
-    }
-  }
-
-  public static String getSegmentId(Configuration hadoopConf) throws IOException {
-    return hadoopConf.get(SEGMENT_ID);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
deleted file mode 100644
index 1e227c4..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
+++ /dev/null
@@ -1,759 +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.carbondata.hadoop.streaming;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.reader.CarbonHeaderReader;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.filter.FilterUtil;
-import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.core.scan.filter.intf.RowImpl;
-import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.format.BlockletHeader;
-import org.apache.carbondata.format.FileHeader;
-import org.apache.carbondata.hadoop.CarbonInputSplit;
-import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
-import org.apache.carbondata.hadoop.InputMetricsStats;
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
-import org.apache.carbondata.hadoop.util.CarbonTypeUtil;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.spark.memory.MemoryMode;
-import org.apache.spark.sql.catalyst.InternalRow;
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
-import org.apache.spark.sql.execution.vectorized.ColumnVector;
-import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
-import org.apache.spark.sql.types.CalendarIntervalType;
-import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.sql.types.DecimalType;
-import org.apache.spark.sql.types.StructType;
-import org.apache.spark.unsafe.types.CalendarInterval;
-import org.apache.spark.unsafe.types.UTF8String;
-
-/**
- * Stream record reader
- */
-public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
-  // vector reader
-  private boolean isVectorReader;
-
-  // metadata
-  private CarbonTable carbonTable;
-  private CarbonColumn[] storageColumns;
-  private boolean[] isRequired;
-  private DataType[] measureDataTypes;
-  private int dimensionCount;
-  private int measureCount;
-
-  // input
-  private FileSplit fileSplit;
-  private Configuration hadoopConf;
-  private StreamBlockletReader input;
-  private boolean isFirstRow = true;
-  private QueryModel model;
-
-  // decode data
-  private BitSet allNonNull;
-  private boolean[] isNoDictColumn;
-  private DirectDictionaryGenerator[] directDictionaryGenerators;
-  private CacheProvider cacheProvider;
-  private Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache;
-  private GenericQueryType[] queryTypes;
-
-  // vectorized reader
-  private StructType outputSchema;
-  private ColumnarBatch columnarBatch;
-  private boolean isFinished = false;
-
-  // filter
-  private FilterExecuter filter;
-  private boolean[] isFilterRequired;
-  private Object[] filterValues;
-  private RowIntf filterRow;
-  private int[] filterMap;
-
-  // output
-  private CarbonColumn[] projection;
-  private boolean[] isProjectionRequired;
-  private int[] projectionMap;
-  private Object[] outputValues;
-  private InternalRow outputRow;
-
-  // empty project, null filter
-  private boolean skipScanData;
-
-  // return raw row for handoff
-  private boolean useRawRow = false;
-
-  // InputMetricsStats
-  private InputMetricsStats inputMetricsStats;
-
-  @Override public void initialize(InputSplit split, TaskAttemptContext context)
-      throws IOException, InterruptedException {
-    // input
-    if (split instanceof CarbonInputSplit) {
-      fileSplit = (CarbonInputSplit) split;
-    } else if (split instanceof CarbonMultiBlockSplit) {
-      fileSplit = ((CarbonMultiBlockSplit) split).getAllSplits().get(0);
-    } else {
-      fileSplit = (FileSplit) split;
-    }
-
-    // metadata
-    hadoopConf = context.getConfiguration();
-    if (model == null) {
-      CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
-      model = format.createQueryModel(split, context);
-    }
-    carbonTable = model.getTable();
-    List<CarbonDimension> dimensions =
-        carbonTable.getDimensionByTableName(carbonTable.getTableName());
-    dimensionCount = dimensions.size();
-    List<CarbonMeasure> measures =
-        carbonTable.getMeasureByTableName(carbonTable.getTableName());
-    measureCount = measures.size();
-    List<CarbonColumn> carbonColumnList =
-        carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName());
-    storageColumns = carbonColumnList.toArray(new CarbonColumn[carbonColumnList.size()]);
-    isNoDictColumn = CarbonDataProcessorUtil.getNoDictionaryMapping(storageColumns);
-    directDictionaryGenerators = new DirectDictionaryGenerator[storageColumns.length];
-    for (int i = 0; i < storageColumns.length; i++) {
-      if (storageColumns[i].hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        directDictionaryGenerators[i] = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(storageColumns[i].getDataType());
-      }
-    }
-    measureDataTypes = new DataType[measureCount];
-    for (int i = 0; i < measureCount; i++) {
-      measureDataTypes[i] = storageColumns[dimensionCount + i].getDataType();
-    }
-
-    // decode data
-    allNonNull = new BitSet(storageColumns.length);
-    projection = model.getProjectionColumns();
-
-    isRequired = new boolean[storageColumns.length];
-    boolean[] isFiltlerDimensions = model.getIsFilterDimensions();
-    boolean[] isFiltlerMeasures = model.getIsFilterMeasures();
-    isFilterRequired = new boolean[storageColumns.length];
-    filterMap = new int[storageColumns.length];
-    for (int i = 0; i < storageColumns.length; i++) {
-      if (storageColumns[i].isDimension()) {
-        if (isFiltlerDimensions[storageColumns[i].getOrdinal()]) {
-          isRequired[i] = true;
-          isFilterRequired[i] = true;
-          filterMap[i] = storageColumns[i].getOrdinal();
-        }
-      } else {
-        if (isFiltlerMeasures[storageColumns[i].getOrdinal()]) {
-          isRequired[i] = true;
-          isFilterRequired[i] = true;
-          filterMap[i] = carbonTable.getDimensionOrdinalMax() + storageColumns[i].getOrdinal();
-        }
-      }
-    }
-
-    isProjectionRequired = new boolean[storageColumns.length];
-    projectionMap = new int[storageColumns.length];
-    for (int i = 0; i < storageColumns.length; i++) {
-      for (int j = 0; j < projection.length; j++) {
-        if (storageColumns[i].getColName().equals(projection[j].getColName())) {
-          isRequired[i] = true;
-          isProjectionRequired[i] = true;
-          projectionMap[i] = j;
-          break;
-        }
-      }
-    }
-
-    // initialize filter
-    if (null != model.getFilterExpressionResolverTree()) {
-      initializeFilter();
-    } else if (projection.length == 0) {
-      skipScanData = true;
-    }
-
-  }
-
-  private void initializeFilter() {
-
-    List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
-        .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
-            carbonTable.getMeasureByTableName(carbonTable.getTableName()));
-    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
-    for (int i = 0; i < dimLensWithComplex.length; i++) {
-      dimLensWithComplex[i] = Integer.MAX_VALUE;
-    }
-
-    int[] dictionaryColumnCardinality =
-        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
-    SegmentProperties segmentProperties =
-        new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality);
-    Map<Integer, GenericQueryType> complexDimensionInfoMap = new HashMap<>();
-
-    FilterResolverIntf resolverIntf = model.getFilterExpressionResolverTree();
-    filter = FilterUtil.getFilterExecuterTree(resolverIntf, segmentProperties,
-        complexDimensionInfoMap);
-    // for row filter, we need update column index
-    FilterUtil.updateIndexOfColumnExpression(resolverIntf.getFilterExpression(),
-        carbonTable.getDimensionOrdinalMax());
-
-  }
-
-  public void setQueryModel(QueryModel model) {
-    this.model = model;
-  }
-
-  private byte[] getSyncMarker(String filePath) throws IOException {
-    CarbonHeaderReader headerReader = new CarbonHeaderReader(filePath);
-    FileHeader header = headerReader.readHeader();
-    return header.getSync_marker();
-  }
-
-  public void setUseRawRow(boolean useRawRow) {
-    this.useRawRow = useRawRow;
-  }
-
-  private void initializeAtFirstRow() throws IOException {
-    filterValues = new Object[carbonTable.getDimensionOrdinalMax() + measureCount];
-    filterRow = new RowImpl();
-    filterRow.setValues(filterValues);
-
-    outputValues = new Object[projection.length];
-    outputRow = new GenericInternalRow(outputValues);
-
-    Path file = fileSplit.getPath();
-
-    byte[] syncMarker = getSyncMarker(file.toString());
-
-    FileSystem fs = file.getFileSystem(hadoopConf);
-
-    int bufferSize = Integer.parseInt(hadoopConf.get(CarbonStreamInputFormat.READ_BUFFER_SIZE,
-        CarbonStreamInputFormat.READ_BUFFER_SIZE_DEFAULT));
-
-    FSDataInputStream fileIn = fs.open(file, bufferSize);
-    fileIn.seek(fileSplit.getStart());
-    input = new StreamBlockletReader(syncMarker, fileIn, fileSplit.getLength(),
-        fileSplit.getStart() == 0);
-
-    cacheProvider = CacheProvider.getInstance();
-    cache = cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
-    queryTypes = CarbonStreamInputFormat.getComplexDimensions(carbonTable, storageColumns, cache);
-
-    outputSchema = new StructType(CarbonTypeUtil.convertCarbonSchemaToSparkSchema(projection));
-  }
-
-  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
-    if (isFirstRow) {
-      isFirstRow = false;
-      initializeAtFirstRow();
-    }
-    if (isFinished) {
-      return false;
-    }
-
-    if (isVectorReader) {
-      return nextColumnarBatch();
-    }
-
-    return nextRow();
-  }
-
-  /**
-   * for vector reader, check next columnar batch
-   */
-  private boolean nextColumnarBatch() throws IOException {
-    boolean hasNext;
-    boolean scanMore = false;
-    do {
-      // move to the next blocklet
-      hasNext = input.nextBlocklet();
-      if (hasNext) {
-        // read blocklet header
-        BlockletHeader header = input.readBlockletHeader();
-        if (isScanRequired(header)) {
-          scanMore = !scanBlockletAndFillVector(header);
-        } else {
-          input.skipBlockletData(true);
-          scanMore = true;
-        }
-      } else {
-        isFinished = true;
-        scanMore = false;
-      }
-    } while (scanMore);
-    return hasNext;
-  }
-
-  /**
-   * check next Row
-   */
-  private boolean nextRow() throws IOException {
-    // read row one by one
-    try {
-      boolean hasNext;
-      boolean scanMore = false;
-      do {
-        hasNext = input.hasNext();
-        if (hasNext) {
-          if (skipScanData) {
-            input.nextRow();
-            scanMore = false;
-          } else {
-            if (useRawRow) {
-              // read raw row for streaming handoff which does not require decode raw row
-              readRawRowFromStream();
-            } else {
-              readRowFromStream();
-            }
-            if (null != filter) {
-              scanMore = !filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax());
-            } else {
-              scanMore = false;
-            }
-          }
-        } else {
-          if (input.nextBlocklet()) {
-            BlockletHeader header = input.readBlockletHeader();
-            if (isScanRequired(header)) {
-              if (skipScanData) {
-                input.skipBlockletData(false);
-              } else {
-                input.readBlockletData(header);
-              }
-            } else {
-              input.skipBlockletData(true);
-            }
-            scanMore = true;
-          } else {
-            isFinished = true;
-            scanMore = false;
-          }
-        }
-      } while (scanMore);
-      return hasNext;
-    } catch (FilterUnsupportedException e) {
-      throw new IOException("Failed to filter row in detail reader", e);
-    }
-  }
-
-  @Override public Void getCurrentKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  @Override public Object getCurrentValue() throws IOException, InterruptedException {
-    if (isVectorReader) {
-      int value = columnarBatch.numValidRows();
-      if (inputMetricsStats != null) {
-        inputMetricsStats.incrementRecordRead((long) value);
-      }
-
-      return columnarBatch;
-    }
-
-    if (inputMetricsStats != null) {
-      inputMetricsStats.incrementRecordRead(1L);
-    }
-
-    return outputRow;
-  }
-
-  private boolean isScanRequired(BlockletHeader header) {
-    // TODO require to implement min-max index
-    if (null == filter) {
-      return true;
-    }
-    return true;
-  }
-
-  private boolean scanBlockletAndFillVector(BlockletHeader header) throws IOException {
-    // if filter is null and output projection is empty, use the row number of blocklet header
-    if (skipScanData) {
-      int rowNums = header.getBlocklet_info().getNum_rows();
-      columnarBatch = ColumnarBatch.allocate(outputSchema, MemoryMode.OFF_HEAP, rowNums);
-      columnarBatch.setNumRows(rowNums);
-      input.skipBlockletData(true);
-      return rowNums > 0;
-    }
-
-    input.readBlockletData(header);
-    columnarBatch = ColumnarBatch.allocate(outputSchema, MemoryMode.OFF_HEAP, input.getRowNums());
-    int rowNum = 0;
-    if (null == filter) {
-      while (input.hasNext()) {
-        readRowFromStream();
-        putRowToColumnBatch(rowNum++);
-      }
-    } else {
-      try {
-        while (input.hasNext()) {
-          readRowFromStream();
-          if (filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax())) {
-            putRowToColumnBatch(rowNum++);
-          }
-        }
-      } catch (FilterUnsupportedException e) {
-        throw new IOException("Failed to filter row in vector reader", e);
-      }
-    }
-    columnarBatch.setNumRows(rowNum);
-    return rowNum > 0;
-  }
-
-  private void readRowFromStream() {
-    input.nextRow();
-    short nullLen = input.readShort();
-    BitSet nullBitSet = allNonNull;
-    if (nullLen > 0) {
-      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
-    }
-    int colCount = 0;
-    // primitive type dimension
-    for (; colCount < isNoDictColumn.length; colCount++) {
-      if (nullBitSet.get(colCount)) {
-        if (isFilterRequired[colCount]) {
-          filterValues[filterMap[colCount]] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
-        }
-        if (isProjectionRequired[colCount]) {
-          outputValues[projectionMap[colCount]] = null;
-        }
-      } else {
-        if (isNoDictColumn[colCount]) {
-          int v = input.readShort();
-          if (isRequired[colCount]) {
-            byte[] b = input.readBytes(v);
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = b;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] =
-                  DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(b,
-                      storageColumns[colCount].getDataType());
-            }
-          } else {
-            input.skipBytes(v);
-          }
-        } else if (null != directDictionaryGenerators[colCount]) {
-          if (isRequired[colCount]) {
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = input.copy(4);
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] =
-                  directDictionaryGenerators[colCount].getValueFromSurrogate(input.readInt());
-            } else {
-              input.skipBytes(4);
-            }
-          } else {
-            input.skipBytes(4);
-          }
-        } else {
-          if (isRequired[colCount]) {
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = input.copy(4);
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = input.readInt();
-            } else {
-              input.skipBytes(4);
-            }
-          } else {
-            input.skipBytes(4);
-          }
-        }
-      }
-    }
-    // complex type dimension
-    for (; colCount < dimensionCount; colCount++) {
-      if (nullBitSet.get(colCount)) {
-        if (isFilterRequired[colCount]) {
-          filterValues[filterMap[colCount]] = null;
-        }
-        if (isProjectionRequired[colCount]) {
-          outputValues[projectionMap[colCount]] = null;
-        }
-      } else {
-        short v = input.readShort();
-        if (isRequired[colCount]) {
-          byte[] b = input.readBytes(v);
-          if (isFilterRequired[colCount]) {
-            filterValues[filterMap[colCount]] = b;
-          }
-          if (isProjectionRequired[colCount]) {
-            outputValues[projectionMap[colCount]] = queryTypes[colCount]
-                .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(b));
-          }
-        } else {
-          input.skipBytes(v);
-        }
-      }
-    }
-    // measure
-    DataType dataType;
-    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
-      if (nullBitSet.get(colCount)) {
-        if (isFilterRequired[colCount]) {
-          filterValues[filterMap[colCount]] = null;
-        }
-        if (isProjectionRequired[colCount]) {
-          outputValues[projectionMap[colCount]] = null;
-        }
-      } else {
-        dataType = measureDataTypes[msrCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          if (isRequired[colCount]) {
-            boolean v = input.readBoolean();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(1);
-          }
-        } else if (dataType == DataTypes.SHORT) {
-          if (isRequired[colCount]) {
-            short v = input.readShort();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(2);
-          }
-        } else if (dataType == DataTypes.INT) {
-          if (isRequired[colCount]) {
-            int v = input.readInt();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(4);
-          }
-        } else if (dataType == DataTypes.LONG) {
-          if (isRequired[colCount]) {
-            long v = input.readLong();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(8);
-          }
-        } else if (dataType == DataTypes.DOUBLE) {
-          if (isRequired[colCount]) {
-            double v = input.readDouble();
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = v;
-            }
-          } else {
-            input.skipBytes(8);
-          }
-        } else if (DataTypes.isDecimal(dataType)) {
-          int len = input.readShort();
-          if (isRequired[colCount]) {
-            BigDecimal v = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
-            if (isFilterRequired[colCount]) {
-              filterValues[filterMap[colCount]] = v;
-            }
-            if (isProjectionRequired[colCount]) {
-              outputValues[projectionMap[colCount]] = Decimal.apply(v);
-            }
-          } else {
-            input.skipBytes(len);
-          }
-        }
-      }
-    }
-  }
-
-  private void readRawRowFromStream() {
-    input.nextRow();
-    short nullLen = input.readShort();
-    BitSet nullBitSet = allNonNull;
-    if (nullLen > 0) {
-      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
-    }
-    int colCount = 0;
-    // primitive type dimension
-    for (; colCount < isNoDictColumn.length; colCount++) {
-      if (nullBitSet.get(colCount)) {
-        outputValues[colCount] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
-      } else {
-        if (isNoDictColumn[colCount]) {
-          int v = input.readShort();
-          outputValues[colCount] = input.readBytes(v);
-        } else {
-          outputValues[colCount] = input.readInt();
-        }
-      }
-    }
-    // complex type dimension
-    for (; colCount < dimensionCount; colCount++) {
-      if (nullBitSet.get(colCount)) {
-        outputValues[colCount] = null;
-      } else {
-        short v = input.readShort();
-        outputValues[colCount] = input.readBytes(v);
-      }
-    }
-    // measure
-    DataType dataType;
-    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
-      if (nullBitSet.get(colCount)) {
-        outputValues[colCount] = null;
-      } else {
-        dataType = measureDataTypes[msrCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          outputValues[colCount] = input.readBoolean();
-        } else if (dataType == DataTypes.SHORT) {
-          outputValues[colCount] = input.readShort();
-        } else if (dataType == DataTypes.INT) {
-          outputValues[colCount] = input.readInt();
-        } else if (dataType == DataTypes.LONG) {
-          outputValues[colCount] = input.readLong();
-        } else if (dataType == DataTypes.DOUBLE) {
-          outputValues[colCount] = input.readDouble();
-        } else if (DataTypes.isDecimal(dataType)) {
-          int len = input.readShort();
-          outputValues[colCount] = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
-        }
-      }
-    }
-  }
-
-  private void putRowToColumnBatch(int rowId) {
-    for (int i = 0; i < projection.length; i++) {
-      Object value = outputValues[i];
-      ColumnVector col = columnarBatch.column(i);
-      org.apache.spark.sql.types.DataType t = col.dataType();
-      if (null == value) {
-        col.putNull(rowId);
-      } else {
-        if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
-          col.putBoolean(rowId, (boolean)value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
-          col.putByte(rowId, (byte) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
-          col.putShort(rowId, (short) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
-          col.putInt(rowId, (int) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
-          col.putLong(rowId, (long) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
-          col.putFloat(rowId, (float) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
-          col.putDouble(rowId, (double) value);
-        } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
-          UTF8String v = (UTF8String) value;
-          col.putByteArray(rowId, v.getBytes());
-        } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
-          DecimalType dt = (DecimalType)t;
-          Decimal d = Decimal.fromDecimal(value);
-          if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
-            col.putInt(rowId, (int)d.toUnscaledLong());
-          } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
-            col.putLong(rowId, d.toUnscaledLong());
-          } else {
-            final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
-            byte[] bytes = integer.toByteArray();
-            col.putByteArray(rowId, bytes, 0, bytes.length);
-          }
-        } else if (t instanceof CalendarIntervalType) {
-          CalendarInterval c = (CalendarInterval) value;
-          col.getChildColumn(0).putInt(rowId, c.months);
-          col.getChildColumn(1).putLong(rowId, c.microseconds);
-        } else if (t instanceof org.apache.spark.sql.types.DateType) {
-          col.putInt(rowId, (int) value);
-        } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
-          col.putLong(rowId, (long) value);
-        }
-      }
-    }
-  }
-
-  @Override public float getProgress() throws IOException, InterruptedException {
-    return 0;
-  }
-
-  public void setVectorReader(boolean isVectorReader) {
-    this.isVectorReader = isVectorReader;
-  }
-
-  public void setInputMetricsStats(InputMetricsStats inputMetricsStats) {
-    this.inputMetricsStats = inputMetricsStats;
-  }
-
-  @Override public void close() throws IOException {
-    if (null != input) {
-      input.close();
-    }
-    if (null != columnarBatch) {
-      columnarBatch.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
deleted file mode 100644
index a4b3be8..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
+++ /dev/null
@@ -1,325 +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.carbondata.hadoop.streaming;
-
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.CarbonMetadataUtil;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.format.FileHeader;
-import org.apache.carbondata.processing.loading.BadRecordsLogger;
-import org.apache.carbondata.processing.loading.BadRecordsLoggerProvider;
-import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.loading.DataField;
-import org.apache.carbondata.processing.loading.DataLoadProcessBuilder;
-import org.apache.carbondata.processing.loading.converter.RowConverter;
-import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl;
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-import org.apache.carbondata.processing.loading.parser.RowParser;
-import org.apache.carbondata.processing.loading.parser.impl.RowParserImpl;
-import org.apache.carbondata.processing.store.writer.AbstractFactDataWriter;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskID;
-
-/**
- * Stream record writer
- */
-public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonStreamRecordWriter.class.getName());
-
-  // basic info
-  private Configuration hadoopConf;
-  private CarbonLoadModel carbonLoadModel;
-  private CarbonDataLoadConfiguration configuration;
-  private CarbonTable carbonTable;
-  private int maxRowNums;
-  private int maxCacheSize;
-
-  // parser and converter
-  private RowParser rowParser;
-  private BadRecordsLogger badRecordLogger;
-  private RowConverter converter;
-  private CarbonRow currentRow = new CarbonRow(null);
-
-  // encoder
-  private DataField[] dataFields;
-  private BitSet nullBitSet;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private int dimensionWithComplexCount;
-  private int measureCount;
-  private DataType[] measureDataTypes;
-  private StreamBlockletWriter output = null;
-
-  // data write
-  private String segmentDir;
-  private String fileName;
-  private DataOutputStream outputStream;
-  private boolean isFirstRow = true;
-  private boolean hasException = false;
-
-  CarbonStreamRecordWriter(TaskAttemptContext job) throws IOException {
-    initialize(job);
-  }
-
-  public CarbonStreamRecordWriter(TaskAttemptContext job, CarbonLoadModel carbonLoadModel)
-      throws IOException {
-    this.carbonLoadModel = carbonLoadModel;
-    initialize(job);
-  }
-
-  private void initialize(TaskAttemptContext job) throws IOException {
-    // set basic information
-    hadoopConf = job.getConfiguration();
-    if (carbonLoadModel == null) {
-      carbonLoadModel = CarbonStreamOutputFormat.getCarbonLoadModel(hadoopConf);
-      if (carbonLoadModel == null) {
-        throw new IOException(
-            "CarbonStreamRecordWriter require configuration: mapreduce.output.carbon.load.model");
-      }
-    }
-    String segmentId = CarbonStreamOutputFormat.getSegmentId(hadoopConf);
-    carbonLoadModel.setSegmentId(segmentId);
-    carbonTable = carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable();
-    long taskNo = TaskID.forName(hadoopConf.get("mapred.tip.id")).getId();
-    carbonLoadModel.setTaskNo("" + taskNo);
-    configuration = DataLoadProcessBuilder.createConfiguration(carbonLoadModel);
-    maxRowNums = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_STREAM_BLOCKLET_ROW_NUMS,
-        CarbonStreamOutputFormat.CARBON_STREAM_BLOCKLET_ROW_NUMS_DEFAULT) - 1;
-    maxCacheSize = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE,
-        CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE_DEFAULT);
-
-    segmentDir = CarbonTablePath.getSegmentPath(
-        carbonTable.getAbsoluteTableIdentifier().getTablePath(), segmentId);
-    fileName = CarbonTablePath.getCarbonDataFileName(0, taskNo, 0, 0, "0");
-  }
-
-  private void initializeAtFirstRow() throws IOException, InterruptedException {
-
-    // initialize metadata
-    isNoDictionaryDimensionColumn =
-        CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
-    dimensionWithComplexCount = configuration.getDimensionCount();
-    measureCount = configuration.getMeasureCount();
-    dataFields = configuration.getDataFields();
-    measureDataTypes = new DataType[measureCount];
-    for (int i = 0; i < measureCount; i++) {
-      measureDataTypes[i] =
-          dataFields[dimensionWithComplexCount + i].getColumn().getDataType();
-    }
-
-    // initialize parser and converter
-    rowParser = new RowParserImpl(dataFields, configuration);
-    badRecordLogger = BadRecordsLoggerProvider.createBadRecordLogger(configuration);
-    converter = new RowConverterImpl(configuration.getDataFields(), configuration, badRecordLogger);
-    configuration.setCardinalityFinder(converter);
-    converter.initialize();
-
-    // initialize encoder
-    nullBitSet = new BitSet(dataFields.length);
-    int rowBufferSize = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_ENCODER_ROW_BUFFER_SIZE,
-        CarbonStreamOutputFormat.CARBON_ENCODER_ROW_BUFFER_SIZE_DEFAULT);
-    output = new StreamBlockletWriter(maxCacheSize, maxRowNums, rowBufferSize);
-
-    // initialize data writer
-    String filePath = segmentDir + File.separator + fileName;
-    FileFactory.FileType fileType = FileFactory.getFileType(filePath);
-    CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, fileType);
-    if (carbonFile.exists()) {
-      // if the file is existed, use the append api
-      outputStream = FileFactory.getDataOutputStreamUsingAppend(filePath, fileType);
-    } else {
-      // IF the file is not existed, use the create api
-      outputStream = FileFactory.getDataOutputStream(filePath, fileType);
-      writeFileHeader();
-    }
-
-    isFirstRow = false;
-  }
-
-  @Override public void write(Void key, Object value) throws IOException, InterruptedException {
-    if (isFirstRow) {
-      initializeAtFirstRow();
-    }
-
-    // parse and convert row
-    currentRow.setData(rowParser.parseRow((Object[]) value));
-    converter.convert(currentRow);
-
-    // null bit set
-    nullBitSet.clear();
-    for (int i = 0; i < dataFields.length; i++) {
-      if (null == currentRow.getObject(i)) {
-        nullBitSet.set(i);
-      }
-    }
-    output.nextRow();
-    byte[] b = nullBitSet.toByteArray();
-    output.writeShort(b.length);
-    if (b.length > 0) {
-      output.writeBytes(b);
-    }
-    int dimCount = 0;
-    Object columnValue;
-
-    // primitive type dimension
-    for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
-      columnValue = currentRow.getObject(dimCount);
-      if (null != columnValue) {
-        if (isNoDictionaryDimensionColumn[dimCount]) {
-          byte[] col = (byte[]) columnValue;
-          output.writeShort(col.length);
-          output.writeBytes(col);
-        } else {
-          output.writeInt((int) columnValue);
-        }
-      }
-    }
-    // complex type dimension
-    for (; dimCount < dimensionWithComplexCount; dimCount++) {
-      columnValue = currentRow.getObject(dimCount);
-      if (null != columnValue) {
-        byte[] col = (byte[]) columnValue;
-        output.writeShort(col.length);
-        output.writeBytes(col);
-      }
-    }
-    // measure
-    DataType dataType;
-    for (int msrCount = 0; msrCount < measureCount; msrCount++) {
-      columnValue = currentRow.getObject(dimCount + msrCount);
-      if (null != columnValue) {
-        dataType = measureDataTypes[msrCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          output.writeBoolean((boolean) columnValue);
-        } else if (dataType == DataTypes.SHORT) {
-          output.writeShort((short) columnValue);
-        } else if (dataType == DataTypes.INT) {
-          output.writeInt((int) columnValue);
-        } else if (dataType == DataTypes.LONG) {
-          output.writeLong((long) columnValue);
-        } else if (dataType == DataTypes.DOUBLE) {
-          output.writeDouble((double) columnValue);
-        } else if (DataTypes.isDecimal(dataType)) {
-          BigDecimal val = (BigDecimal) columnValue;
-          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
-          output.writeShort(bigDecimalInBytes.length);
-          output.writeBytes(bigDecimalInBytes);
-        } else {
-          String msg =
-              "unsupported data type:" + dataFields[dimCount + msrCount].getColumn().getDataType()
-                  .getName();
-          LOGGER.error(msg);
-          throw new IOException(msg);
-        }
-      }
-    }
-
-    if (output.isFull()) {
-      appendBlockletToDataFile();
-    }
-  }
-
-  private void writeFileHeader() throws IOException {
-    List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
-        .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
-            carbonTable.getMeasureByTableName(carbonTable.getTableName()));
-    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
-    for (int i = 0; i < dimLensWithComplex.length; i++) {
-      dimLensWithComplex[i] = Integer.MAX_VALUE;
-    }
-    int[] dictionaryColumnCardinality =
-        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
-    List<Integer> cardinality = new ArrayList<>();
-    List<org.apache.carbondata.format.ColumnSchema> columnSchemaList = AbstractFactDataWriter
-        .getColumnSchemaListAndCardinality(cardinality, dictionaryColumnCardinality,
-            wrapperColumnSchemaList);
-    FileHeader fileHeader =
-        CarbonMetadataUtil.getFileHeader(true, columnSchemaList, System.currentTimeMillis());
-    fileHeader.setIs_footer_present(false);
-    fileHeader.setIs_splitable(true);
-    fileHeader.setSync_marker(CarbonStreamOutputFormat.CARBON_SYNC_MARKER);
-    outputStream.write(CarbonUtil.getByteArray(fileHeader));
-  }
-
-  /**
-   * write a blocklet to file
-   */
-  private void appendBlockletToDataFile() throws IOException {
-    if (output.getRowIndex() == -1) {
-      return;
-    }
-    output.apppendBlocklet(outputStream);
-    outputStream.flush();
-    // reset data
-    output.reset();
-  }
-
-  @Override public void close(TaskAttemptContext context) throws IOException, InterruptedException {
-    try {
-      // append remain buffer data
-      if (!hasException && !isFirstRow) {
-        appendBlockletToDataFile();
-        converter.finish();
-      }
-    } finally {
-      // close resource
-      CarbonUtil.closeStreams(outputStream);
-      if (output != null) {
-        output.close();
-      }
-      if (badRecordLogger != null) {
-        badRecordLogger.closeStreams();
-      }
-    }
-  }
-
-  public String getSegmentDir() {
-    return segmentDir;
-  }
-
-  public String getFileName() {
-    return fileName;
-  }
-
-  public void setHasException(boolean hasException) {
-    this.hasException = hasException;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletReader.java
deleted file mode 100644
index 1989198..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletReader.java
+++ /dev/null
@@ -1,259 +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.carbondata.hadoop.streaming;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.apache.carbondata.core.datastore.compression.Compressor;
-import org.apache.carbondata.core.datastore.compression.CompressorFactory;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.format.BlockletHeader;
-
-/**
- * stream blocklet reader
- */
-public class StreamBlockletReader {
-
-  private byte[] buffer;
-  private int offset;
-  private final byte[] syncMarker;
-  private final byte[] syncBuffer;
-  private final int syncLen;
-  private long pos = 0;
-  private final InputStream in;
-  private final long limitStart;
-  private final long limitEnd;
-  private boolean isAlreadySync = false;
-  private Compressor compressor = CompressorFactory.getInstance().getCompressor();
-  private int rowNums = 0;
-  private int rowIndex = 0;
-  private boolean isHeaderPresent;
-
-  StreamBlockletReader(byte[] syncMarker, InputStream in, long limit, boolean isHeaderPresent) {
-    this.syncMarker = syncMarker;
-    syncLen = syncMarker.length;
-    syncBuffer = new byte[syncLen];
-    this.in = in;
-    limitStart = limit;
-    limitEnd = limitStart + syncLen;
-    this.isHeaderPresent = isHeaderPresent;
-  }
-
-  private void ensureCapacity(int capacity) {
-    if (buffer == null || capacity > buffer.length) {
-      buffer = new byte[capacity];
-    }
-  }
-
-  /**
-   * find the first position of sync_marker in input stream
-   */
-  private boolean sync() throws IOException {
-    if (!readBytesFromStream(syncBuffer, 0, syncLen)) {
-      return false;
-    }
-    boolean skipHeader = false;
-    for (int i = 0; i < limitStart; i++) {
-      int j = 0;
-      for (; j < syncLen; j++) {
-        if (syncMarker[j] != syncBuffer[(i + j) % syncLen]) break;
-      }
-      if (syncLen == j) {
-        if (isHeaderPresent) {
-          if (skipHeader) {
-            return true;
-          } else {
-            skipHeader = true;
-          }
-        } else {
-          return true;
-        }
-      }
-      int value = in.read();
-      if (-1 == value) {
-        return false;
-      }
-      syncBuffer[i % syncLen] = (byte) value;
-      pos++;
-    }
-    return false;
-  }
-
-  BlockletHeader readBlockletHeader() throws IOException {
-    int len = readIntFromStream();
-    byte[] b = new byte[len];
-    if (!readBytesFromStream(b, 0, len)) {
-      throw new EOFException("Failed to read blocklet header");
-    }
-    BlockletHeader header = CarbonUtil.readBlockletHeader(b);
-    rowNums = header.getBlocklet_info().getNum_rows();
-    rowIndex = 0;
-    return header;
-  }
-
-  void readBlockletData(BlockletHeader header) throws IOException {
-    ensureCapacity(header.getBlocklet_length());
-    offset = 0;
-    int len = readIntFromStream();
-    byte[] b = new byte[len];
-    if (!readBytesFromStream(b, 0, len)) {
-      throw new EOFException("Failed to read blocklet data");
-    }
-    compressor.rawUncompress(b, buffer);
-  }
-
-  void skipBlockletData(boolean reset) throws IOException {
-    int len = readIntFromStream();
-    skip(len);
-    pos += len;
-    if (reset) {
-      this.rowNums = 0;
-      this.rowIndex = 0;
-    }
-  }
-
-  private void skip(int len) throws IOException {
-    long remaining = len;
-    do {
-      long skipLen = in.skip(remaining);
-      remaining -= skipLen;
-    } while (remaining > 0);
-  }
-
-  /**
-   * find the next blocklet
-   */
-  boolean nextBlocklet() throws IOException {
-    if (pos >= limitStart) {
-      return false;
-    }
-    if (isAlreadySync) {
-      if (!readBytesFromStream(syncBuffer, 0, syncLen)) {
-        return false;
-      }
-    } else {
-      isAlreadySync = true;
-      if (!sync()) {
-        return false;
-      }
-    }
-
-    return pos < limitEnd;
-  }
-
-  boolean hasNext() throws IOException {
-    return rowIndex < rowNums;
-  }
-
-  void nextRow() {
-    rowIndex++;
-  }
-
-  int readIntFromStream() throws IOException {
-    int ch1 = in.read();
-    int ch2 = in.read();
-    int ch3 = in.read();
-    int ch4 = in.read();
-    if ((ch1 | ch2 | ch3 | ch4) < 0) throw new EOFException();
-    pos += 4;
-    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
-  }
-
-  /**
-   * Reads <code>len</code> bytes of data from the input stream into
-   * an array of bytes.
-   * @return <code>true</code> if reading data successfully, or
-   * <code>false</code> if there is no more data because the end of the stream has been reached.
-   */
-  boolean readBytesFromStream(byte[] b, int offset, int len) throws IOException {
-    int readLen = in.read(b, offset, len);
-    if (readLen < 0) {
-      return false;
-    }
-    pos += readLen;
-    if (readLen < len) {
-      return readBytesFromStream(b, offset + readLen, len - readLen);
-    } else {
-      return true;
-    }
-  }
-
-  boolean readBoolean() {
-    return (buffer[offset++]) != 0;
-  }
-
-  short readShort() {
-    short v =  (short) ((buffer[offset + 1] & 255) +
-        ((buffer[offset]) << 8));
-    offset += 2;
-    return v;
-  }
-
-  byte[] copy(int len) {
-    byte[] b = new byte[len];
-    System.arraycopy(buffer, offset, b, 0, len);
-    return b;
-  }
-
-  int readInt() {
-    int v = ((buffer[offset + 3] & 255) +
-        ((buffer[offset + 2] & 255) << 8) +
-        ((buffer[offset + 1] & 255) << 16) +
-        ((buffer[offset]) << 24));
-    offset += 4;
-    return v;
-  }
-
-  long readLong() {
-    long v = ((long)(buffer[offset + 7] & 255)) +
-        ((long) (buffer[offset + 6] & 255) << 8) +
-        ((long) (buffer[offset + 5] & 255) << 16) +
-        ((long) (buffer[offset + 4] & 255) << 24) +
-        ((long) (buffer[offset + 3] & 255) << 32) +
-        ((long) (buffer[offset + 2] & 255) << 40) +
-        ((long) (buffer[offset + 1] & 255) << 48) +
-        ((long) (buffer[offset]) << 56);
-    offset += 8;
-    return v;
-  }
-
-  double readDouble() {
-    return Double.longBitsToDouble(readLong());
-  }
-
-  byte[] readBytes(int len) {
-    byte[] b = new byte[len];
-    System.arraycopy(buffer, offset, b, 0, len);
-    offset += len;
-    return b;
-  }
-
-  void skipBytes(int len) {
-    offset += len;
-  }
-
-  int getRowNums() {
-    return rowNums;
-  }
-
-  void close() {
-    CarbonUtil.closeStreams(in);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletWriter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletWriter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletWriter.java
deleted file mode 100644
index a0328b3..0000000
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/StreamBlockletWriter.java
+++ /dev/null
@@ -1,152 +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.carbondata.hadoop.streaming;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.carbondata.core.datastore.compression.Compressor;
-import org.apache.carbondata.core.datastore.compression.CompressorFactory;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.format.BlockletHeader;
-import org.apache.carbondata.format.BlockletInfo;
-import org.apache.carbondata.format.MutationType;
-
-/**
- * stream blocklet writer
- */
-public class StreamBlockletWriter {
-  private byte[] buffer;
-  private int maxSize;
-  private int maxRowNum;
-  private int rowSize;
-  private int count = 0;
-  private int rowIndex = -1;
-  private Compressor compressor = CompressorFactory.getInstance().getCompressor();
-
-  StreamBlockletWriter(int maxSize, int maxRowNum, int rowSize) {
-    buffer = new byte[maxSize];
-    this.maxSize = maxSize;
-    this.maxRowNum = maxRowNum;
-    this.rowSize = rowSize;
-  }
-
-  private void ensureCapacity(int space) {
-    int newcount = space + count;
-    if (newcount > buffer.length) {
-      byte[] newbuf = new byte[Math.max(newcount, buffer.length + rowSize)];
-      System.arraycopy(buffer, 0, newbuf, 0, count);
-      buffer = newbuf;
-    }
-  }
-
-  void reset() {
-    count = 0;
-    rowIndex = -1;
-  }
-
-  byte[] getBytes() {
-    return buffer;
-  }
-
-  int getCount() {
-    return count;
-  }
-
-  int getRowIndex() {
-    return rowIndex;
-  }
-
-  void nextRow() {
-    rowIndex++;
-  }
-
-  boolean isFull() {
-    return rowIndex == maxRowNum || count >= maxSize;
-  }
-
-  void writeBoolean(boolean val) {
-    ensureCapacity(1);
-    buffer[count] = (byte) (val ? 1 : 0);
-    count += 1;
-  }
-
-  void writeShort(int val) {
-    ensureCapacity(2);
-    buffer[count + 1] = (byte) (val);
-    buffer[count] = (byte) (val >>> 8);
-    count += 2;
-  }
-
-  void writeInt(int val) {
-    ensureCapacity(4);
-    buffer[count + 3] = (byte) (val);
-    buffer[count + 2] = (byte) (val >>> 8);
-    buffer[count + 1] = (byte) (val >>> 16);
-    buffer[count] = (byte) (val >>> 24);
-    count += 4;
-  }
-
-  void writeLong(long val) {
-    ensureCapacity(8);
-    buffer[count + 7] = (byte) (val);
-    buffer[count + 6] = (byte) (val >>> 8);
-    buffer[count + 5] = (byte) (val >>> 16);
-    buffer[count + 4] = (byte) (val >>> 24);
-    buffer[count + 3] = (byte) (val >>> 32);
-    buffer[count + 2] = (byte) (val >>> 40);
-    buffer[count + 1] = (byte) (val >>> 48);
-    buffer[count] = (byte) (val >>> 56);
-    count += 8;
-  }
-
-  void writeDouble(double val) {
-    writeLong(Double.doubleToLongBits(val));
-  }
-
-  void writeBytes(byte[] b) {
-    writeBytes(b, 0, b.length);
-  }
-
-  void writeBytes(byte[] b, int off, int len) {
-    ensureCapacity(len);
-    System.arraycopy(b, off, buffer, count, len);
-    count += len;
-  }
-
-  void apppendBlocklet(DataOutputStream outputStream) throws IOException {
-    outputStream.write(CarbonStreamOutputFormat.CARBON_SYNC_MARKER);
-
-    BlockletInfo blockletInfo = new BlockletInfo();
-    blockletInfo.setNum_rows(getRowIndex() + 1);
-    BlockletHeader blockletHeader = new BlockletHeader();
-    blockletHeader.setBlocklet_length(getCount());
-    blockletHeader.setMutation(MutationType.INSERT);
-    blockletHeader.setBlocklet_info(blockletInfo);
-    byte[] headerBytes = CarbonUtil.getByteArray(blockletHeader);
-    outputStream.writeInt(headerBytes.length);
-    outputStream.write(headerBytes);
-
-    byte[] compressed = compressor.compressByte(getBytes(), getCount());
-    outputStream.writeInt(compressed.length);
-    outputStream.write(compressed);
-  }
-
-  void close() {
-  }
-}


[3/5] carbondata git commit: [CARBONDATA-2165]Remove spark in carbon-hadoop module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
new file mode 100644
index 0000000..bf5f660
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
@@ -0,0 +1,435 @@
+/*
+ * 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.carbondata.spark.rdd
+
+import java.text.SimpleDateFormat
+import java.util
+import java.util.Date
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.{Job, TaskAttemptID, TaskType}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.datamap.Segment
+import org.apache.carbondata.core.datastore.block.SegmentProperties
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.scan.result.iterator.RawResultIterator
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
+import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonProjection}
+import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
+import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostStatusUpdateEvent, LoadTablePreStatusUpdateEvent}
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.merger.{CompactionResultSortProcessor, CompactionType}
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
+import org.apache.carbondata.spark.{HandoffResult, HandoffResultImpl}
+import org.apache.carbondata.spark.util.CommonUtil
+import org.apache.carbondata.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
+
+
+/**
+ * partition of the handoff segment
+ */
+class HandoffPartition(
+    val rddId: Int,
+    val idx: Int,
+    @transient val inputSplit: CarbonInputSplit
+) extends Partition {
+
+  val split = new SerializableWritable[CarbonInputSplit](inputSplit)
+
+  override val index: Int = idx
+
+  override def hashCode(): Int = 41 * (41 + rddId) + idx
+}
+
+/**
+ * package the record reader of the handoff segment to RawResultIterator
+ */
+class StreamingRawResultIterator(
+    recordReader: CarbonStreamRecordReader
+) extends RawResultIterator(null, null, null) {
+
+  override def hasNext: Boolean = {
+    recordReader.nextKeyValue()
+  }
+
+  override def next(): Array[Object] = {
+    val rowTmp = recordReader
+      .getCurrentValue
+      .asInstanceOf[GenericInternalRow]
+      .values
+      .asInstanceOf[Array[Object]]
+    val row = new Array[Object](rowTmp.length)
+    System.arraycopy(rowTmp, 0, row, 0, rowTmp.length)
+    row
+  }
+}
+
+/**
+ * execute streaming segment handoff
+ */
+class StreamHandoffRDD[K, V](
+    sc: SparkContext,
+    result: HandoffResult[K, V],
+    carbonLoadModel: CarbonLoadModel,
+    handOffSegmentId: String
+) extends CarbonRDD[(K, V)](sc, Nil, sc.hadoopConfiguration) {
+
+  private val jobTrackerId: String = {
+    val formatter = new SimpleDateFormat("yyyyMMddHHmm")
+    formatter.format(new Date())
+  }
+
+  override def internalCompute(
+      split: Partition,
+      context: TaskContext
+  ): Iterator[(K, V)] = {
+    carbonLoadModel.setTaskNo("" + split.index)
+    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    CarbonMetadata.getInstance().addCarbonTable(carbonTable)
+    // the input iterator is using raw row
+    val iteratorList = prepareInputIterator(split, carbonTable)
+
+    CommonUtil.setTempStoreLocation(split.index, carbonLoadModel, true, false)
+    // use CompactionResultSortProcessor to sort data dan write to columnar files
+    val processor = prepareHandoffProcessor(carbonTable)
+    val status = processor.execute(iteratorList)
+
+    new Iterator[(K, V)] {
+      private var finished = false
+
+      override def hasNext: Boolean = {
+        !finished
+      }
+
+      override def next(): (K, V) = {
+        finished = true
+        result.getKey("" + split.index, status)
+      }
+    }
+  }
+
+  /**
+   * prepare input iterator by basing CarbonStreamRecordReader
+   */
+  private def prepareInputIterator(
+      split: Partition,
+      carbonTable: CarbonTable
+  ): util.ArrayList[RawResultIterator] = {
+    val inputSplit = split.asInstanceOf[HandoffPartition].split.value
+    val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0)
+    val hadoopConf = new Configuration()
+    CarbonInputFormat.setDatabaseName(hadoopConf, carbonTable.getDatabaseName)
+    CarbonInputFormat.setTableName(hadoopConf, carbonTable.getTableName)
+    CarbonInputFormat.setTablePath(hadoopConf, carbonTable.getTablePath)
+    val projection = new CarbonProjection
+    val dataFields = carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName)
+    (0 until dataFields.size()).foreach { index =>
+      projection.addColumn(dataFields.get(index).getColName)
+    }
+    CarbonInputFormat.setColumnProjection(hadoopConf, projection)
+    CarbonInputFormat.setTableInfo(hadoopConf, carbonTable.getTableInfo)
+    val attemptContext = new TaskAttemptContextImpl(hadoopConf, attemptId)
+    val format = new CarbonTableInputFormat[Array[Object]]()
+    val model = format.createQueryModel(inputSplit, attemptContext)
+    val inputFormat = new CarbonStreamInputFormat
+    val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
+      .asInstanceOf[CarbonStreamRecordReader]
+    streamReader.setVectorReader(false)
+    streamReader.setQueryModel(model)
+    streamReader.setUseRawRow(true)
+    streamReader.initialize(inputSplit, attemptContext)
+    val iteratorList = new util.ArrayList[RawResultIterator](1)
+    iteratorList.add(new StreamingRawResultIterator(streamReader))
+    iteratorList
+  }
+
+  private def prepareHandoffProcessor(
+      carbonTable: CarbonTable
+  ): CompactionResultSortProcessor = {
+    val wrapperColumnSchemaList = CarbonUtil.getColumnSchemaList(
+      carbonTable.getDimensionByTableName(carbonTable.getTableName),
+      carbonTable.getMeasureByTableName(carbonTable.getTableName))
+    val dimLensWithComplex =
+      (0 until wrapperColumnSchemaList.size()).map(_ => Integer.MAX_VALUE).toArray
+    val dictionaryColumnCardinality =
+      CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList)
+    val segmentProperties =
+      new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality)
+
+    new CompactionResultSortProcessor(
+      carbonLoadModel,
+      carbonTable,
+      segmentProperties,
+      CompactionType.STREAMING,
+      carbonTable.getTableName,
+      null
+    )
+  }
+
+  /**
+   * get the partitions of the handoff segment
+   */
+  override protected def getPartitions: Array[Partition] = {
+    val job = Job.getInstance(FileFactory.getConfiguration)
+    val inputFormat = new CarbonTableInputFormat[Array[Object]]()
+    val segmentList = new util.ArrayList[Segment](1)
+    segmentList.add(Segment.toSegment(handOffSegmentId))
+    val splits = inputFormat.getSplitsOfStreaming(
+      job,
+      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getAbsoluteTableIdentifier,
+      segmentList
+    )
+
+    (0 until splits.size()).map { index =>
+      new HandoffPartition(id, index, splits.get(index).asInstanceOf[CarbonInputSplit])
+    }.toArray[Partition]
+  }
+}
+
+object StreamHandoffRDD {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def iterateStreamingHandoff(
+      carbonLoadModel: CarbonLoadModel,
+      sparkSession: SparkSession
+  ): Unit = {
+    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    val identifier = carbonTable.getAbsoluteTableIdentifier
+    var continueHandoff = false
+    // require handoff lock on table
+    val lock = CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.HANDOFF_LOCK)
+    try {
+      if (lock.lockWithRetries()) {
+        LOGGER.info("Acquired the handoff lock for table" +
+                    s" ${ carbonTable.getDatabaseName }.${ carbonTable.getTableName }")
+        // handoff streaming segment one by one
+        do {
+          val segmentStatusManager = new SegmentStatusManager(identifier)
+          var loadMetadataDetails: Array[LoadMetadataDetails] = null
+          // lock table to read table status file
+          val statusLock = segmentStatusManager.getTableStatusLock
+          try {
+            if (statusLock.lockWithRetries()) {
+              loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
+                CarbonTablePath.getMetadataPath(identifier.getTablePath))
+            }
+          } finally {
+            if (null != statusLock) {
+              statusLock.unlock()
+            }
+          }
+          if (null != loadMetadataDetails) {
+            val streamSegments =
+              loadMetadataDetails.filter(_.getSegmentStatus == SegmentStatus.STREAMING_FINISH)
+
+            continueHandoff = streamSegments.length > 0
+            if (continueHandoff) {
+              // handoff a streaming segment
+              val loadMetadataDetail = streamSegments(0)
+              executeStreamingHandoff(
+                carbonLoadModel,
+                sparkSession,
+                loadMetadataDetail.getLoadName
+              )
+            }
+          } else {
+            continueHandoff = false
+          }
+        } while (continueHandoff)
+      }
+    } finally {
+      if (null != lock) {
+        lock.unlock()
+      }
+    }
+  }
+
+  /**
+   * start new thread to execute stream segment handoff
+   */
+  def startStreamingHandoffThread(
+      carbonLoadModel: CarbonLoadModel,
+      sparkSession: SparkSession,
+      isDDL: Boolean
+  ): Unit = {
+    if (isDDL) {
+      iterateStreamingHandoff(carbonLoadModel, sparkSession)
+    } else {
+      // start a new thread to execute streaming segment handoff
+      val handoffThread = new Thread() {
+        override def run(): Unit = {
+          iterateStreamingHandoff(carbonLoadModel, sparkSession)
+        }
+      }
+      handoffThread.start()
+    }
+  }
+
+  /**
+   * invoke StreamHandoffRDD to handoff a streaming segment to a columnar segment
+   */
+  def executeStreamingHandoff(
+      carbonLoadModel: CarbonLoadModel,
+      sparkSession: SparkSession,
+      handoffSegmenId: String
+  ): Unit = {
+    var loadStatus = SegmentStatus.SUCCESS
+    var errorMessage: String = "Handoff failure"
+    try {
+      // generate new columnar segment
+      val newMetaEntry = new LoadMetadataDetails
+      carbonLoadModel.setFactTimeStamp(System.currentTimeMillis())
+      CarbonLoaderUtil.populateNewLoadMetaEntry(
+        newMetaEntry,
+        SegmentStatus.INSERT_IN_PROGRESS,
+        carbonLoadModel.getFactTimeStamp,
+        false)
+      val operationContext = new OperationContext()
+      val loadTablePreStatusUpdateEvent: LoadTablePreStatusUpdateEvent =
+        new LoadTablePreStatusUpdateEvent(
+          carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getCarbonTableIdentifier,
+          carbonLoadModel)
+      OperationListenerBus.getInstance().fireEvent(loadTablePreStatusUpdateEvent, operationContext)
+
+      CarbonLoaderUtil.recordNewLoadMetadata(newMetaEntry, carbonLoadModel, true, false)
+      val loadTablePostStatusUpdateEvent: LoadTablePostStatusUpdateEvent =
+        new LoadTablePostStatusUpdateEvent(carbonLoadModel)
+      OperationListenerBus.getInstance()
+        .fireEvent(loadTablePostStatusUpdateEvent, operationContext)
+      // convert a streaming segment to columnar segment
+      val status = new StreamHandoffRDD(
+        sparkSession.sparkContext,
+        new HandoffResultImpl(),
+        carbonLoadModel,
+        handoffSegmenId).collect()
+
+      status.foreach { x =>
+        if (!x._2) {
+          loadStatus = SegmentStatus.LOAD_FAILURE
+        }
+      }
+    } catch {
+      case ex: Exception =>
+        loadStatus = SegmentStatus.LOAD_FAILURE
+        errorMessage = errorMessage + ": " + ex.getCause.getMessage
+        LOGGER.error(errorMessage)
+        LOGGER.error(ex, s"Handoff failed on streaming segment $handoffSegmenId")
+    }
+
+    if (loadStatus == SegmentStatus.LOAD_FAILURE) {
+      CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel)
+      LOGGER.info("********starting clean up**********")
+      CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
+      LOGGER.info("********clean up done**********")
+      LOGGER.audit(s"Handoff is failed for " +
+                   s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      LOGGER.error("Cannot write load metadata file as handoff failed")
+      throw new Exception(errorMessage)
+    }
+
+    if (loadStatus == SegmentStatus.SUCCESS) {
+      val done = updateLoadMetadata(handoffSegmenId, carbonLoadModel)
+      if (!done) {
+        val errorMessage = "Handoff failed due to failure in table status updation."
+        LOGGER.audit("Handoff is failed for " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        LOGGER.error("Handoff failed due to failure in table status updation.")
+        throw new Exception(errorMessage)
+      }
+      done
+    }
+
+  }
+
+  /**
+   * update streaming segment and new columnar segment
+   */
+  private def updateLoadMetadata(
+      handoffSegmentId: String,
+      loadModel: CarbonLoadModel
+  ): Boolean = {
+    var status = false
+    val metaDataFilepath = loadModel.getCarbonDataLoadSchema.getCarbonTable.getMetadataPath
+    val identifier = loadModel.getCarbonDataLoadSchema.getCarbonTable.getAbsoluteTableIdentifier
+    val metadataPath = CarbonTablePath.getMetadataPath(identifier.getTablePath)
+    val fileType = FileFactory.getFileType(metadataPath)
+    if (!FileFactory.isFileExist(metadataPath, fileType)) {
+      FileFactory.mkdirs(metadataPath, fileType)
+    }
+    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath)
+    val segmentStatusManager = new SegmentStatusManager(identifier)
+    val carbonLock = segmentStatusManager.getTableStatusLock
+    try {
+      if (carbonLock.lockWithRetries()) {
+        LOGGER.info(
+          "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
+          + " for table status updation")
+        val listOfLoadFolderDetailsArray =
+          SegmentStatusManager.readLoadMetadata(metaDataFilepath)
+
+        // update new columnar segment to success status
+        val newSegment =
+          listOfLoadFolderDetailsArray.find(_.getLoadName.equals(loadModel.getSegmentId))
+        if (newSegment.isEmpty) {
+          throw new Exception("Failed to update table status for new segment")
+        } else {
+          newSegment.get.setSegmentStatus(SegmentStatus.SUCCESS)
+          newSegment.get.setLoadEndTime(System.currentTimeMillis())
+        }
+
+        // update streaming segment to compacted status
+        val streamSegment =
+          listOfLoadFolderDetailsArray.find(_.getLoadName.equals(handoffSegmentId))
+        if (streamSegment.isEmpty) {
+          throw new Exception("Failed to update table status for streaming segment")
+        } else {
+          streamSegment.get.setSegmentStatus(SegmentStatus.COMPACTED)
+          streamSegment.get.setMergedLoadName(loadModel.getSegmentId)
+        }
+
+        // refresh table status file
+        SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetailsArray)
+        status = true
+      } else {
+        LOGGER.error("Not able to acquire the lock for Table status updation for table " + loadModel
+          .getDatabaseName() + "." + loadModel.getTableName())
+      }
+    } finally {
+      if (carbonLock.unlock()) {
+        LOGGER.info("Table unlocked successfully after table status updation" +
+                    loadModel.getDatabaseName() + "." + loadModel.getTableName())
+      } else {
+        LOGGER.error("Unable to unlock Table lock for table" + loadModel.getDatabaseName() +
+                     "." + loadModel.getTableName() + " during table status updation")
+      }
+    }
+    status
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index 3250a53..5f55ef3 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -47,6 +47,7 @@ import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+import org.apache.carbondata.streaming.parser.FieldConverter
 
 object CarbonScalaUtil {
 
@@ -121,55 +122,8 @@ object CarbonScalaUtil {
       timeStampFormat: SimpleDateFormat,
       dateFormat: SimpleDateFormat,
       level: Int = 1): String = {
-    if (value == null) {
-      serializationNullFormat
-    } else {
-      value match {
-        case s: String => if (s.length > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
-          throw new Exception("Dataload failed, String length cannot exceed " +
-                              CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " characters")
-        } else {
-          s
-        }
-        case d: java.math.BigDecimal => d.toPlainString
-        case i: java.lang.Integer => i.toString
-        case d: java.lang.Double => d.toString
-        case t: java.sql.Timestamp => timeStampFormat format t
-        case d: java.sql.Date => dateFormat format d
-        case b: java.lang.Boolean => b.toString
-        case s: java.lang.Short => s.toString
-        case f: java.lang.Float => f.toString
-        case bs: Array[Byte] => new String(bs,
-          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET))
-        case s: scala.collection.Seq[Any] =>
-          val delimiter = if (level == 1) {
-            delimiterLevel1
-          } else {
-            delimiterLevel2
-          }
-          val builder = new StringBuilder()
-          s.foreach { x =>
-            builder.append(getString(x, serializationNullFormat, delimiterLevel1,
-                delimiterLevel2, timeStampFormat, dateFormat, level + 1)).append(delimiter)
-          }
-          builder.substring(0, builder.length - delimiter.length())
-        case m: scala.collection.Map[Any, Any] =>
-          throw new Exception("Unsupported data type: Map")
-        case r: org.apache.spark.sql.Row =>
-          val delimiter = if (level == 1) {
-            delimiterLevel1
-          } else {
-            delimiterLevel2
-          }
-          val builder = new StringBuilder()
-          for (i <- 0 until r.length) {
-            builder.append(getString(r(i), serializationNullFormat, delimiterLevel1,
-                delimiterLevel2, timeStampFormat, dateFormat, level + 1)).append(delimiter)
-          }
-          builder.substring(0, builder.length - delimiter.length())
-        case other => other.toString
-      }
-    }
+    FieldConverter.objectToString(value, serializationNullFormat, delimiterLevel1,
+      delimiterLevel2, timeStampFormat, dateFormat, level)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonSparkStreamingListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonSparkStreamingListener.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonSparkStreamingListener.scala
new file mode 100644
index 0000000..a99a1e8
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonSparkStreamingListener.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.carbondata.streaming
+
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+
+class CarbonSparkStreamingListener extends SparkListener {
+
+  /**
+   * When Spark Streaming App stops, remove all locks for stream table.
+   */
+  override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+    CarbonStreamSparkStreaming.cleanAllLockAfterStop()
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonStreamSparkStreaming.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonStreamSparkStreaming.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonStreamSparkStreaming.scala
new file mode 100644
index 0000000..28f04b1
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonStreamSparkStreaming.scala
@@ -0,0 +1,184 @@
+/*
+ * 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.carbondata.streaming
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
+import org.apache.spark.sql.execution.streaming.{CarbonAppendableStreamSink, Sink}
+import org.apache.spark.streaming.Time
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+
+/**
+ * Interface used to write stream data to stream table
+ * when integrate with Spark Streaming.
+ *
+ * NOTE: Current integration with Spark Streaming is an alpha feature.
+ */
+class CarbonStreamSparkStreamingWriter(val sparkSession: SparkSession,
+    val carbonTable: CarbonTable,
+    val configuration: Configuration) {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  private var isInitialize: Boolean = false
+
+  private var lock: ICarbonLock = null
+  private var carbonAppendableStreamSink: Sink = null
+
+  /**
+   * Acquired the lock for stream table
+   */
+  def lockStreamTable(): Unit = {
+    lock = CarbonLockFactory.getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
+      LockUsage.STREAMING_LOCK)
+    if (lock.lockWithRetries()) {
+      LOGGER.info("Acquired the lock for stream table: " +
+                  carbonTable.getDatabaseName + "." +
+                  carbonTable.getTableName)
+    } else {
+      LOGGER.error("Not able to acquire the lock for stream table:" +
+                   carbonTable.getDatabaseName + "." + carbonTable.getTableName)
+      throw new InterruptedException(
+        "Not able to acquire the lock for stream table: " + carbonTable.getDatabaseName + "." +
+        carbonTable.getTableName)
+    }
+  }
+
+  /**
+   * unlock for stream table
+   */
+  def unLockStreamTable(): Unit = {
+    if (null != lock) {
+      lock.unlock()
+      LOGGER.info("unlock for stream table: " +
+                  carbonTable.getDatabaseName + "." +
+                  carbonTable.getTableName)
+    }
+  }
+
+  def initialize(): Unit = {
+    carbonAppendableStreamSink = StreamSinkFactory.createStreamTableSink(
+      sparkSession,
+      configuration,
+      carbonTable,
+      extraOptions.toMap).asInstanceOf[CarbonAppendableStreamSink]
+
+    lockStreamTable()
+
+    isInitialize = true
+  }
+
+  def writeStreamData(dataFrame: DataFrame, time: Time): Unit = {
+    if (!isInitialize) {
+      initialize()
+    }
+    carbonAppendableStreamSink.addBatch(time.milliseconds, dataFrame)
+  }
+
+  private val extraOptions = new scala.collection.mutable.HashMap[String, String]
+  private var mode: SaveMode = SaveMode.ErrorIfExists
+
+  this.option("dbName", carbonTable.getDatabaseName)
+  this.option("tableName", carbonTable.getTableName)
+
+  /**
+   * Specifies the behavior when data or table already exists. Options include:
+   *   - `SaveMode.Overwrite`: overwrite the existing data.
+   *   - `SaveMode.Append`: append the data.
+   *   - `SaveMode.Ignore`: ignore the operation (i.e. no-op).
+   *   - `SaveMode.ErrorIfExists`: default option, throw an exception at runtime.
+   */
+  def mode(saveMode: SaveMode): CarbonStreamSparkStreamingWriter = {
+    if (mode == SaveMode.ErrorIfExists) {
+      mode = saveMode
+    }
+    this
+  }
+
+  /**
+   * Specifies the behavior when data or table already exists. Options include:
+   *   - `overwrite`: overwrite the existing data.
+   *   - `append`: append the data.
+   *   - `ignore`: ignore the operation (i.e. no-op).
+   *   - `error or default`: default option, throw an exception at runtime.
+   */
+  def mode(saveMode: String): CarbonStreamSparkStreamingWriter = {
+    if (mode == SaveMode.ErrorIfExists) {
+      mode = saveMode.toLowerCase(util.Locale.ROOT) match {
+        case "overwrite" => SaveMode.Overwrite
+        case "append" => SaveMode.Append
+        case "ignore" => SaveMode.Ignore
+        case "error" | "default" => SaveMode.ErrorIfExists
+        case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. " +
+          "Accepted save modes are 'overwrite', 'append', 'ignore', 'error'.")
+      }
+    }
+    this
+  }
+
+  /**
+   * Adds an output option
+   */
+  def option(key: String, value: String): CarbonStreamSparkStreamingWriter = {
+    if (!extraOptions.contains(key)) {
+      extraOptions += (key -> value)
+    }
+    this
+  }
+
+  /**
+   * Adds an output option
+   */
+  def option(key: String, value: Boolean): CarbonStreamSparkStreamingWriter =
+    option(key, value.toString)
+
+  /**
+   * Adds an output option
+   */
+  def option(key: String, value: Long): CarbonStreamSparkStreamingWriter =
+    option(key, value.toString)
+
+  /**
+   * Adds an output option
+   */
+  def option(key: String, value: Double): CarbonStreamSparkStreamingWriter =
+    option(key, value.toString)
+}
+
+object CarbonStreamSparkStreaming {
+
+  @transient private val tableMap =
+    new util.HashMap[String, CarbonStreamSparkStreamingWriter]()
+
+  def getTableMap: util.Map[String, CarbonStreamSparkStreamingWriter] = tableMap
+
+  /**
+   * remove all stream lock.
+   */
+  def cleanAllLockAfterStop(): Unit = {
+    tableMap.asScala.values.foreach { writer => writer.unLockStreamTable() }
+    tableMap.clear()
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonStreamingQueryListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonStreamingQueryListener.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonStreamingQueryListener.scala
new file mode 100644
index 0000000..6d83fad
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/CarbonStreamingQueryListener.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.carbondata.streaming
+
+import java.util
+import java.util.UUID
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.{CarbonAppendableStreamSink, StreamExecution}
+import org.apache.spark.sql.streaming.StreamingQueryListener
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
+
+class CarbonStreamingQueryListener(spark: SparkSession) extends StreamingQueryListener {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  private val cache = new util.HashMap[UUID, ICarbonLock]()
+
+  override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {
+    val streamQuery = spark.streams.get(event.id)
+    val qry = if (streamQuery.isInstanceOf[StreamExecution]) {
+      // adapt spark 2.1
+      streamQuery.asInstanceOf[StreamExecution]
+    } else {
+      // adapt spark 2.2 and later version
+      val clazz = Class.forName("org.apache.spark.sql.execution.streaming.StreamingQueryWrapper")
+      val method = clazz.getMethod("streamingQuery")
+      method.invoke(streamQuery).asInstanceOf[StreamExecution]
+    }
+    if (qry.sink.isInstanceOf[CarbonAppendableStreamSink]) {
+      LOGGER.info("Carbon streaming query started: " + event.id)
+      val sink = qry.sink.asInstanceOf[CarbonAppendableStreamSink]
+      val carbonTable = sink.carbonTable
+      val lock = CarbonLockFactory.getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
+        LockUsage.STREAMING_LOCK)
+      if (lock.lockWithRetries()) {
+        LOGGER.info("Acquired the lock for stream table: " + carbonTable.getDatabaseName + "." +
+                    carbonTable.getTableName)
+        cache.put(event.id, lock)
+      } else {
+        LOGGER.error("Not able to acquire the lock for stream table:" +
+                     carbonTable.getDatabaseName + "." + carbonTable.getTableName)
+        throw new InterruptedException(
+          "Not able to acquire the lock for stream table: " + carbonTable.getDatabaseName + "." +
+          carbonTable.getTableName)
+      }
+    }
+  }
+
+  override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {
+  }
+
+  override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {
+    val lock = cache.remove(event.id)
+    if (null != lock) {
+      LOGGER.info("Carbon streaming query: " + event.id)
+      lock.unlock()
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
new file mode 100644
index 0000000..bc7b042
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -0,0 +1,236 @@
+/*
+ * 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.carbondata.streaming
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.streaming.{CarbonAppendableStreamSink, Sink}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.dictionary.server.{DictionaryServer, NonSecureDictionaryServer}
+import org.apache.carbondata.core.dictionary.service.NonSecureDictionaryServiceProvider
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
+import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
+import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
+import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
+import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
+import org.apache.carbondata.streaming.segment.StreamSegment
+
+/**
+ * Stream sink factory
+ */
+object StreamSinkFactory {
+
+  def createStreamTableSink(
+      sparkSession: SparkSession,
+      hadoopConf: Configuration,
+      carbonTable: CarbonTable,
+      parameters: Map[String, String]): Sink = {
+    validateParameters(parameters)
+
+    // build load model
+    val carbonLoadModel = buildCarbonLoadModelForStream(
+      sparkSession,
+      hadoopConf,
+      carbonTable,
+      parameters,
+      "")
+    // fire pre event before streamin is started
+    // in case of streaming options and optionsFinal can be same
+    val operationContext = new OperationContext
+    val loadTablePreExecutionEvent = new LoadTablePreExecutionEvent(
+      carbonTable.getCarbonTableIdentifier,
+      carbonLoadModel,
+      carbonLoadModel.getFactFilePath,
+      false,
+      parameters.asJava,
+      parameters.asJava,
+      false
+    )
+    OperationListenerBus.getInstance().fireEvent(loadTablePreExecutionEvent, operationContext)
+    // prepare the stream segment
+    val segmentId = getStreamSegmentId(carbonTable)
+    carbonLoadModel.setSegmentId(segmentId)
+
+    // start server if necessary
+    val server = startDictionaryServer(
+      sparkSession,
+      carbonTable,
+      carbonLoadModel)
+    if (server.isDefined) {
+      carbonLoadModel.setUseOnePass(true)
+    } else {
+      carbonLoadModel.setUseOnePass(false)
+    }
+    // default is carbon appended stream sink
+    val carbonAppendableStreamSink = new CarbonAppendableStreamSink(
+      sparkSession,
+      carbonTable,
+      segmentId,
+      parameters,
+      carbonLoadModel,
+      server)
+
+    // fire post event before streamin is started
+    val loadTablePostExecutionEvent = new LoadTablePostExecutionEvent(
+      carbonTable.getCarbonTableIdentifier,
+      carbonLoadModel
+    )
+    OperationListenerBus.getInstance().fireEvent(loadTablePostExecutionEvent, operationContext)
+    carbonAppendableStreamSink
+  }
+
+  private def validateParameters(parameters: Map[String, String]): Unit = {
+    val segmentSize = parameters.get(CarbonCommonConstants.HANDOFF_SIZE)
+    if (segmentSize.isDefined) {
+      try {
+        val value = java.lang.Long.parseLong(segmentSize.get)
+        if (value < CarbonCommonConstants.HANDOFF_SIZE_MIN) {
+          new CarbonStreamException(CarbonCommonConstants.HANDOFF_SIZE +
+                                    "should be bigger than or equal " +
+                                    CarbonCommonConstants.HANDOFF_SIZE_MIN)
+        }
+      } catch {
+        case _: NumberFormatException =>
+          new CarbonStreamException(CarbonCommonConstants.HANDOFF_SIZE +
+                                    s" $segmentSize is an illegal number")
+      }
+    }
+  }
+
+  /**
+   * get current stream segment id
+   * @return
+   */
+  private def getStreamSegmentId(carbonTable: CarbonTable): String = {
+    val segmentId = StreamSegment.open(carbonTable)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
+    val fileType = FileFactory.getFileType(segmentDir)
+    if (!FileFactory.isFileExist(segmentDir, fileType)) {
+      // Create table directory path, in case of enabling hive metastore first load may not have
+      // table folder created.
+      FileFactory.mkdirs(segmentDir, fileType)
+    }
+    if (FileFactory.isFileExist(segmentDir, fileType)) {
+      // recover fault
+      StreamSegment.recoverSegmentIfRequired(segmentDir)
+    } else {
+      FileFactory.mkdirs(segmentDir, fileType)
+    }
+    segmentId
+  }
+
+  def startDictionaryServer(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      carbonLoadModel: CarbonLoadModel): Option[DictionaryServer] = {
+    // start dictionary server when use one pass load and dimension with DICTIONARY
+    // encoding is present.
+    val allDimensions = carbonTable.getAllDimensions.asScala.toList
+    val createDictionary = allDimensions.exists {
+      carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
+                         !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
+    }
+    val carbonSecureModeDictServer = CarbonProperties.getInstance.
+      getProperty(CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER,
+        CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER_DEFAULT)
+
+    val sparkConf = sparkSession.sqlContext.sparkContext.getConf
+    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
+      getConf.get("spark.driver.host")
+
+    val server: Option[DictionaryServer] = if (createDictionary) {
+      if (sparkConf.get("spark.authenticate", "false").equalsIgnoreCase("true") &&
+          carbonSecureModeDictServer.toBoolean) {
+        val dictionaryServer = SecureDictionaryServer.getInstance(sparkConf,
+          sparkDriverHost.toString, carbonLoadModel.getDictionaryServerPort, carbonTable)
+        carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
+        carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
+        carbonLoadModel.setDictionaryServerSecretKey(dictionaryServer.getSecretKey)
+        carbonLoadModel.setDictionaryEncryptServerSecure(dictionaryServer.isEncryptSecureServer)
+        carbonLoadModel.setDictionaryServiceProvider(new SecureDictionaryServiceProvider())
+        sparkSession.sparkContext.addSparkListener(new SparkListener() {
+          override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
+            dictionaryServer.shutdown()
+          }
+        })
+        Some(dictionaryServer)
+      } else {
+        val dictionaryServer = NonSecureDictionaryServer
+          .getInstance(carbonLoadModel.getDictionaryServerPort, carbonTable)
+        carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
+        carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
+        carbonLoadModel.setDictionaryEncryptServerSecure(false)
+        carbonLoadModel
+          .setDictionaryServiceProvider(new NonSecureDictionaryServiceProvider(dictionaryServer
+            .getPort))
+        sparkSession.sparkContext.addSparkListener(new SparkListener() {
+          override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
+            dictionaryServer.shutdown()
+          }
+        })
+        Some(dictionaryServer)
+      }
+    } else {
+      None
+    }
+    server
+  }
+
+  private def buildCarbonLoadModelForStream(
+      sparkSession: SparkSession,
+      hadoopConf: Configuration,
+      carbonTable: CarbonTable,
+      parameters: Map[String, String],
+      segmentId: String): CarbonLoadModel = {
+    val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
+    carbonProperty.addProperty("zookeeper.enable.lock", "false")
+    val optionsFinal = LoadOption.fillOptionWithDefaultValue(parameters.asJava)
+    optionsFinal.put("sort_scope", "no_sort")
+    if (parameters.get("fileheader").isEmpty) {
+      optionsFinal.put("fileheader", carbonTable.getCreateOrderColumn(carbonTable.getTableName)
+        .asScala.map(_.getColName).mkString(","))
+    }
+    val carbonLoadModel = new CarbonLoadModel()
+    new CarbonLoadModelBuilder(carbonTable).build(
+      parameters.asJava,
+      optionsFinal,
+      carbonLoadModel,
+      hadoopConf)
+    carbonLoadModel.setSegmentId(segmentId)
+    // stream should use one pass
+    val dictionaryServerPort = parameters.getOrElse(
+      CarbonCommonConstants.DICTIONARY_SERVER_PORT,
+      carbonProperty.getProperty(
+        CarbonCommonConstants.DICTIONARY_SERVER_PORT,
+        CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT))
+    val sparkDriverHost = sparkSession.sqlContext.sparkContext.
+      getConf.get("spark.driver.host")
+    carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
+    carbonLoadModel.setDictionaryServerPort(dictionaryServerPort.toInt)
+    carbonLoadModel
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
new file mode 100644
index 0000000..402bc4b
--- /dev/null
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -0,0 +1,362 @@
+/*
+ * 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.spark.sql.execution.streaming
+
+import java.util.Date
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce._
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.io.FileCommitProtocol
+import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage
+import org.apache.spark.sql.{DataFrame, SparkSession}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.{QueryExecution, SQLExecution}
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.{SerializableConfiguration, Utils}
+
+import org.apache.carbondata.common.CarbonIterator
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.dictionary.server.DictionaryServer
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.stats.QueryStatistic
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
+import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.spark.rdd.StreamHandoffRDD
+import org.apache.carbondata.streaming.{CarbonStreamException, CarbonStreamOutputFormat}
+import org.apache.carbondata.streaming.parser.CarbonStreamParser
+import org.apache.carbondata.streaming.segment.StreamSegment
+
+/**
+ * an implement of stream sink, it persist each batch to disk by appending the batch data to
+ * data files.
+ */
+class CarbonAppendableStreamSink(
+    sparkSession: SparkSession,
+    val carbonTable: CarbonTable,
+    var currentSegmentId: String,
+    parameters: Map[String, String],
+    carbonLoadModel: CarbonLoadModel,
+    server: Option[DictionaryServer]) extends Sink {
+
+  private val fileLogPath = CarbonTablePath.getStreamingLogDir(carbonTable.getTablePath)
+  private val fileLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, fileLogPath)
+  // prepare configuration
+  private val hadoopConf = {
+    val conf = sparkSession.sessionState.newHadoopConf()
+    // put all parameters into hadoopConf
+    parameters.foreach { entry =>
+      conf.set(entry._1, entry._2)
+    }
+    // properties below will be used for default CarbonStreamParser
+    conf.set("carbon_complex_delimiter_level_1",
+      carbonLoadModel.getComplexDelimiterLevel1)
+    conf.set("carbon_complex_delimiter_level_2",
+      carbonLoadModel.getComplexDelimiterLevel2)
+    conf.set(
+      DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT,
+      carbonLoadModel.getSerializationNullFormat().split(",")(1))
+    conf.set(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+      carbonLoadModel.getTimestampformat())
+    conf.set(
+      CarbonCommonConstants.CARBON_DATE_FORMAT,
+      carbonLoadModel.getDateFormat())
+    conf
+  }
+  // segment max size(byte)
+  private val segmentMaxSize = hadoopConf.getLong(
+    CarbonCommonConstants.HANDOFF_SIZE,
+    CarbonProperties.getInstance().getHandoffSize
+  )
+
+  // auto handoff
+  private val enableAutoHandoff = hadoopConf.getBoolean(
+    CarbonCommonConstants.ENABLE_AUTO_HANDOFF,
+    CarbonProperties.getInstance().isEnableAutoHandoff
+  )
+
+  override def addBatch(batchId: Long, data: DataFrame): Unit = {
+    if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) {
+      CarbonAppendableStreamSink.LOGGER.info(s"Skipping already committed batch $batchId")
+    } else {
+
+      val statistic = new QueryStatistic()
+
+      // fire pre event on every batch add
+      // in case of streaming options and optionsFinal can be same
+      val operationContext = new OperationContext
+      val loadTablePreExecutionEvent = new LoadTablePreExecutionEvent(
+        carbonTable.getCarbonTableIdentifier,
+        carbonLoadModel,
+        carbonLoadModel.getFactFilePath,
+        false,
+        parameters.asJava,
+        parameters.asJava,
+        false
+      )
+      OperationListenerBus.getInstance().fireEvent(loadTablePreExecutionEvent, operationContext)
+      checkOrHandOffSegment()
+
+      // committer will record how this spark job commit its output
+      val committer = FileCommitProtocol.instantiate(
+        className = sparkSession.sessionState.conf.streamingFileCommitProtocolClass,
+        jobId = batchId.toString,
+        outputPath = fileLogPath,
+        isAppend = false)
+
+      committer match {
+        case manifestCommitter: ManifestFileCommitProtocol =>
+          manifestCommitter.setupManifestOptions(fileLog, batchId)
+        case _ => // Do nothing
+      }
+
+      CarbonAppendableStreamSink.writeDataFileJob(
+        sparkSession,
+        carbonTable,
+        parameters,
+        batchId,
+        currentSegmentId,
+        data.queryExecution,
+        committer,
+        hadoopConf,
+        carbonLoadModel,
+        server)
+      // fire post event on every batch add
+      val loadTablePostExecutionEvent = new LoadTablePostExecutionEvent(
+        carbonTable.getCarbonTableIdentifier,
+        carbonLoadModel
+      )
+      OperationListenerBus.getInstance().fireEvent(loadTablePostExecutionEvent, operationContext)
+
+      statistic.addStatistics(s"add batch: $batchId", System.currentTimeMillis())
+      CarbonAppendableStreamSink.LOGGER.info(
+        s"${statistic.getMessage}, taken time(ms): ${statistic.getTimeTaken}")
+    }
+  }
+
+  /**
+   * if the directory size of current segment beyond the threshold, hand off new segment
+   */
+  private def checkOrHandOffSegment(): Unit = {
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
+    val fileType = FileFactory.getFileType(segmentDir)
+    if (segmentMaxSize <= StreamSegment.size(segmentDir)) {
+      val newSegmentId = StreamSegment.close(carbonTable, currentSegmentId)
+      currentSegmentId = newSegmentId
+      val newSegmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
+      FileFactory.mkdirs(newSegmentDir, fileType)
+
+      // TODO trigger hand off operation
+      if (enableAutoHandoff) {
+        StreamHandoffRDD.startStreamingHandoffThread(
+          carbonLoadModel,
+          sparkSession,
+          false)
+      }
+    }
+  }
+}
+
+object CarbonAppendableStreamSink {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  /**
+   * package the hadoop configuration and it will be passed to executor side from driver side
+   */
+  case class WriteDataFileJobDescription(
+      serializableHadoopConf: SerializableConfiguration,
+      batchId: Long,
+      segmentId: String)
+
+  /**
+   * Run a spark job to append the newly arrived data to the existing row format
+   * file directly.
+   * If there are failure in the task, spark will re-try the task and
+   * carbon will do recovery by HDFS file truncate. (see StreamSegment.tryRecoverFromTaskFault)
+   * If there are job level failure, every files in the stream segment will do truncate
+   * if necessary. (see StreamSegment.tryRecoverFromJobFault)
+   */
+  def writeDataFileJob(
+      sparkSession: SparkSession,
+      carbonTable: CarbonTable,
+      parameters: Map[String, String],
+      batchId: Long,
+      segmentId: String,
+      queryExecution: QueryExecution,
+      committer: FileCommitProtocol,
+      hadoopConf: Configuration,
+      carbonLoadModel: CarbonLoadModel,
+      server: Option[DictionaryServer]): Unit = {
+
+    // create job
+    val job = Job.getInstance(hadoopConf)
+    job.setOutputKeyClass(classOf[Void])
+    job.setOutputValueClass(classOf[InternalRow])
+    val jobId = CarbonInputFormatUtil.getJobId(new Date, batchId.toInt)
+    job.setJobID(jobId)
+
+    val description = WriteDataFileJobDescription(
+      serializableHadoopConf = new SerializableConfiguration(job.getConfiguration),
+      batchId,
+      segmentId
+    )
+
+    // run write data file job
+    SQLExecution.withNewExecutionId(sparkSession, queryExecution) {
+      var result: Array[TaskCommitMessage] = null
+      try {
+        committer.setupJob(job)
+        // initialize dictionary server
+        if (server.isDefined) {
+          server.get.initializeDictionaryGenerator(carbonTable)
+        }
+
+        val rowSchema = queryExecution.analyzed.schema
+        // write data file
+        result = sparkSession.sparkContext.runJob(queryExecution.toRdd,
+          (taskContext: TaskContext, iterator: Iterator[InternalRow]) => {
+            writeDataFileTask(
+              description,
+              carbonLoadModel,
+              sparkStageId = taskContext.stageId(),
+              sparkPartitionId = taskContext.partitionId(),
+              sparkAttemptNumber = taskContext.attemptNumber(),
+              committer,
+              iterator,
+              rowSchema
+            )
+          })
+
+        // write dictionary
+        if (server.isDefined) {
+          try {
+            server.get.writeTableDictionary(carbonTable.getCarbonTableIdentifier.getTableId)
+          } catch {
+            case _: Exception =>
+              LOGGER.error(
+                s"Error while writing dictionary file for ${carbonTable.getTableUniqueName}")
+              throw new Exception(
+                "Streaming ingest failed due to error while writing dictionary file")
+          }
+        }
+
+        // update data file info in index file
+        StreamSegment.updateIndexFile(
+          CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId))
+
+      } catch {
+        // catch fault of executor side
+        case t: Throwable =>
+          val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
+          StreamSegment.recoverSegmentIfRequired(segmentDir)
+          LOGGER.error(t, s"Aborting job ${ job.getJobID }.")
+          committer.abortJob(job)
+          throw new CarbonStreamException("Job failed to write data file", t)
+      }
+      committer.commitJob(job, result)
+      LOGGER.info(s"Job ${ job.getJobID } committed.")
+    }
+  }
+
+  /**
+   * execute a task for each partition to write a data file
+   */
+  def writeDataFileTask(
+      description: WriteDataFileJobDescription,
+      carbonLoadModel: CarbonLoadModel,
+      sparkStageId: Int,
+      sparkPartitionId: Int,
+      sparkAttemptNumber: Int,
+      committer: FileCommitProtocol,
+      iterator: Iterator[InternalRow],
+      rowSchema: StructType
+  ): TaskCommitMessage = {
+
+    val jobId = CarbonInputFormatUtil.getJobId(new Date, sparkStageId)
+    val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId)
+    val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber)
+
+    // Set up the attempt context required to use in the output committer.
+    val taskAttemptContext: TaskAttemptContext = {
+      // Set up the configuration object
+      val hadoopConf = description.serializableHadoopConf.value
+      CarbonStreamOutputFormat.setSegmentId(hadoopConf, description.segmentId)
+      hadoopConf.set("mapred.job.id", jobId.toString)
+      hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID.toString)
+      hadoopConf.set("mapred.task.id", taskAttemptId.toString)
+      hadoopConf.setBoolean("mapred.task.is.map", true)
+      hadoopConf.setInt("mapred.task.partition", 0)
+      new TaskAttemptContextImpl(hadoopConf, taskAttemptId)
+    }
+
+    committer.setupTask(taskAttemptContext)
+
+    try {
+      Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+
+        val parserName = taskAttemptContext.getConfiguration.get(
+          CarbonStreamParser.CARBON_STREAM_PARSER,
+          CarbonStreamParser.CARBON_STREAM_PARSER_DEFAULT)
+
+        val streamParser =
+          Class.forName(parserName).newInstance.asInstanceOf[CarbonStreamParser]
+        streamParser.initialize(taskAttemptContext.getConfiguration, rowSchema)
+
+        StreamSegment.appendBatchData(new InputIterator(iterator, streamParser),
+          taskAttemptContext, carbonLoadModel)
+      })(catchBlock = {
+        committer.abortTask(taskAttemptContext)
+        LOGGER.error(s"Job $jobId aborted.")
+      })
+      committer.commitTask(taskAttemptContext)
+    } catch {
+      case t: Throwable =>
+        throw new CarbonStreamException("Task failed while writing rows", t)
+    }
+  }
+
+  /**
+   * convert spark iterator to carbon iterator, so that java module can use it.
+   */
+  class InputIterator(rddIter: Iterator[InternalRow], streamParser: CarbonStreamParser)
+    extends CarbonIterator[Array[Object]] {
+
+    override def hasNext: Boolean = rddIter.hasNext
+
+    override def next: Array[Object] = {
+      streamParser.parserRow(rddIter.next())
+    }
+
+    override def close(): Unit = {
+      streamParser.close()
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
index f582145..34f901f 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
@@ -124,7 +124,9 @@ object TestQueryExecutor {
     TestQueryExecutor.projectPath + "/processing/target",
     TestQueryExecutor.projectPath + "/integration/spark-common/target",
     TestQueryExecutor.projectPath + "/integration/spark2/target",
-    TestQueryExecutor.projectPath + "/integration/spark-common/target/jars")
+    TestQueryExecutor.projectPath + "/integration/spark-common/target/jars",
+    TestQueryExecutor.projectPath + "/streaming/target")
+
   lazy val jars = {
     val jarsLocal = new ArrayBuffer[String]()
     modules.foreach { path =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index 13b3d8d..e4593be 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -36,7 +36,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
-      <artifactId>carbondata-streaming</artifactId>
+      <artifactId>carbondata-spark-common</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
index f6bdff6..1038fcf 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
@@ -27,7 +27,6 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
 import org.apache.spark.sql.SparkSession.Builder
 import org.apache.spark.sql.catalyst.encoders.RowEncoder
 import org.apache.spark.sql.catalyst.plans.logical.{Command, LocalRelation, Union}
-import org.apache.spark.sql.execution.streaming.CarbonStreamingQueryListener
 import org.apache.spark.sql.hive.execution.command.CarbonSetCommand
 import org.apache.spark.sql.internal.{SessionState, SharedState}
 import org.apache.spark.sql.profiler.{Profiler, SQLStart}
@@ -36,6 +35,7 @@ import org.apache.spark.util.{CarbonReflectionUtils, Utils}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, ThreadLocalSessionInfo}
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
+import org.apache.carbondata.streaming.CarbonStreamingQueryListener
 
 /**
  * Session implementation for {org.apache.spark.sql.SparkSession}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 5183b02..e60a583 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -46,8 +46,7 @@ import org.apache.carbondata.events._
 import org.apache.carbondata.processing.loading.events.LoadEvents.LoadMetadataEvent
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
-import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
-import org.apache.carbondata.streaming.StreamHandoffRDD
+import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, StreamHandoffRDD}
 import org.apache.carbondata.streaming.segment.StreamSegment
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 01affec..b8c447d 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -22,11 +22,16 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
-      <artifactId>carbondata-spark-common</artifactId>
+      <artifactId>carbondata-hadoop</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming_${scala.binary.version}</artifactId>
       <version>${spark.version}</version>
       <scope>${spark.deps.scope}</scope>
@@ -44,7 +49,7 @@
   </dependencies>
 
   <build>
-    <testSourceDirectory>src/test/scala</testSourceDirectory>
+    <testSourceDirectory>src/test/java</testSourceDirectory>
     <resources>
       <resource>
         <directory>src/resources</directory>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
new file mode 100644
index 0000000..66d89c8
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamInputFormat.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.streaming;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.scan.complextypes.ArrayQueryType;
+import org.apache.carbondata.core.scan.complextypes.PrimitiveQueryType;
+import org.apache.carbondata.core.scan.complextypes.StructQueryType;
+import org.apache.carbondata.core.scan.filter.GenericQueryType;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+/**
+ * Stream input format
+ */
+public class CarbonStreamInputFormat extends FileInputFormat<Void, Object> {
+
+  public static final String READ_BUFFER_SIZE = "carbon.stream.read.buffer.size";
+  public static final String READ_BUFFER_SIZE_DEFAULT = "65536";
+
+  @Override public RecordReader<Void, Object> createRecordReader(InputSplit split,
+      TaskAttemptContext context) throws IOException, InterruptedException {
+    return new CarbonStreamRecordReader();
+  }
+
+  public static GenericQueryType[] getComplexDimensions(CarbonTable carbontable,
+      CarbonColumn[] carbonColumns, Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache)
+      throws IOException {
+    GenericQueryType[] queryTypes = new GenericQueryType[carbonColumns.length];
+    for (int i = 0; i < carbonColumns.length; i++) {
+      if (carbonColumns[i].isComplex()) {
+        if (DataTypes.isArrayType(carbonColumns[i].getDataType())) {
+          queryTypes[i] = new ArrayQueryType(carbonColumns[i].getColName(),
+              carbonColumns[i].getColName(), i);
+        } else if (DataTypes.isStructType(carbonColumns[i].getDataType())) {
+          queryTypes[i] = new StructQueryType(carbonColumns[i].getColName(),
+              carbonColumns[i].getColName(), i);
+        } else {
+          throw new UnsupportedOperationException(
+              carbonColumns[i].getDataType().getName() + " is not supported");
+        }
+
+        fillChildren(carbontable, queryTypes[i], (CarbonDimension) carbonColumns[i], i, cache);
+      }
+    }
+
+    return queryTypes;
+  }
+
+  private static void fillChildren(CarbonTable carbontable, GenericQueryType parentQueryType,
+      CarbonDimension dimension, int parentBlockIndex,
+      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache) throws IOException {
+    for (int i = 0; i < dimension.getNumberOfChild(); i++) {
+      CarbonDimension child = dimension.getListOfChildDimensions().get(i);
+      DataType dataType = child.getDataType();
+      GenericQueryType queryType = null;
+      if (DataTypes.isArrayType(dataType)) {
+        queryType =
+            new ArrayQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
+
+      } else if (DataTypes.isStructType(dataType)) {
+        queryType =
+            new StructQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex);
+        parentQueryType.addChildren(queryType);
+      } else {
+        boolean isDirectDictionary =
+            CarbonUtil.hasEncoding(child.getEncoder(), Encoding.DIRECT_DICTIONARY);
+        String dictionaryPath = carbontable.getTableInfo().getFactTable().getTableProperties()
+            .get(CarbonCommonConstants.DICTIONARY_PATH);
+        DictionaryColumnUniqueIdentifier dictionarIdentifier =
+            new DictionaryColumnUniqueIdentifier(carbontable.getAbsoluteTableIdentifier(),
+                child.getColumnIdentifier(), child.getDataType(), dictionaryPath);
+
+        queryType =
+            new PrimitiveQueryType(child.getColName(), dimension.getColName(), ++parentBlockIndex,
+                child.getDataType(), 4, cache.get(dictionarIdentifier),
+                isDirectDictionary);
+      }
+      parentQueryType.addChildren(queryType);
+      if (child.getNumberOfChild() > 0) {
+        fillChildren(carbontable, queryType, child, parentBlockIndex, cache);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamOutputFormat.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamOutputFormat.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamOutputFormat.java
new file mode 100644
index 0000000..f9f0d76
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamOutputFormat.java
@@ -0,0 +1,87 @@
+/*
+ * 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.carbondata.streaming;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+/**
+ * Stream output format
+ */
+public class CarbonStreamOutputFormat extends FileOutputFormat<Void, Object> {
+
+  static final byte[] CARBON_SYNC_MARKER =
+      "@carbondata_sync".getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+
+  public static final String CARBON_ENCODER_ROW_BUFFER_SIZE = "carbon.stream.row.buffer.size";
+
+  public static final int CARBON_ENCODER_ROW_BUFFER_SIZE_DEFAULT = 1024;
+
+  public static final String CARBON_STREAM_BLOCKLET_ROW_NUMS = "carbon.stream.blocklet.row.nums";
+
+  public static final int CARBON_STREAM_BLOCKLET_ROW_NUMS_DEFAULT = 32000;
+
+  public static final String CARBON_STREAM_CACHE_SIZE = "carbon.stream.cache.size";
+
+  public static final int CARBON_STREAM_CACHE_SIZE_DEFAULT = 32 * 1024 * 1024;
+
+  private static final String LOAD_Model = "mapreduce.output.carbon.load.model";
+
+  private static final String SEGMENT_ID = "carbon.segment.id";
+
+  @Override public RecordWriter<Void, Object> getRecordWriter(TaskAttemptContext job)
+      throws IOException, InterruptedException {
+    return new CarbonStreamRecordWriter(job);
+  }
+
+  public static void setCarbonLoadModel(Configuration hadoopConf, CarbonLoadModel carbonLoadModel)
+      throws IOException {
+    if (carbonLoadModel != null) {
+      hadoopConf.set(LOAD_Model, ObjectSerializationUtil.convertObjectToString(carbonLoadModel));
+    }
+  }
+
+  public static CarbonLoadModel getCarbonLoadModel(Configuration hadoopConf) throws IOException {
+    String value = hadoopConf.get(LOAD_Model);
+    if (value == null) {
+      return null;
+    } else {
+      return (CarbonLoadModel) ObjectSerializationUtil.convertStringToObject(value);
+    }
+  }
+
+  public static void setSegmentId(Configuration hadoopConf, String segmentId) throws IOException {
+    if (segmentId != null) {
+      hadoopConf.set(SEGMENT_ID, segmentId);
+    }
+  }
+
+  public static String getSegmentId(Configuration hadoopConf) throws IOException {
+    return hadoopConf.get(SEGMENT_ID);
+  }
+
+}


[2/5] carbondata git commit: [CARBONDATA-2165]Remove spark in carbon-hadoop module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordReader.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordReader.java
new file mode 100644
index 0000000..69d2a3b
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordReader.java
@@ -0,0 +1,761 @@
+/*
+ * 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.carbondata.streaming;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.reader.CarbonHeaderReader;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.GenericQueryType;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.intf.RowImpl;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.format.BlockletHeader;
+import org.apache.carbondata.format.FileHeader;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+import org.apache.carbondata.hadoop.InputMetricsStats;
+import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.spark.memory.MemoryMode;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.execution.vectorized.ColumnVector;
+import org.apache.spark.sql.execution.vectorized.ColumnarBatch;
+import org.apache.spark.sql.types.CalendarIntervalType;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.sql.types.DecimalType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.CalendarInterval;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Stream record reader
+ */
+public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
+  // vector reader
+  private boolean isVectorReader;
+
+  // metadata
+  private CarbonTable carbonTable;
+  private CarbonColumn[] storageColumns;
+  private boolean[] isRequired;
+  private DataType[] measureDataTypes;
+  private int dimensionCount;
+  private int measureCount;
+
+  // input
+  private FileSplit fileSplit;
+  private Configuration hadoopConf;
+  private StreamBlockletReader input;
+  private boolean isFirstRow = true;
+  private QueryModel model;
+
+  // decode data
+  private BitSet allNonNull;
+  private boolean[] isNoDictColumn;
+  private DirectDictionaryGenerator[] directDictionaryGenerators;
+  private CacheProvider cacheProvider;
+  private Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache;
+  private GenericQueryType[] queryTypes;
+
+  // vectorized reader
+  private StructType outputSchema;
+  private ColumnarBatch columnarBatch;
+  private boolean isFinished = false;
+
+  // filter
+  private FilterExecuter filter;
+  private boolean[] isFilterRequired;
+  private Object[] filterValues;
+  private RowIntf filterRow;
+  private int[] filterMap;
+
+  // output
+  private CarbonColumn[] projection;
+  private boolean[] isProjectionRequired;
+  private int[] projectionMap;
+  private Object[] outputValues;
+  private InternalRow outputRow;
+
+  // empty project, null filter
+  private boolean skipScanData;
+
+  // return raw row for handoff
+  private boolean useRawRow = false;
+
+  // InputMetricsStats
+  private InputMetricsStats inputMetricsStats;
+
+  @Override public void initialize(InputSplit split, TaskAttemptContext context)
+      throws IOException, InterruptedException {
+    // input
+    if (split instanceof CarbonInputSplit) {
+      fileSplit = (CarbonInputSplit) split;
+    } else if (split instanceof CarbonMultiBlockSplit) {
+      fileSplit = ((CarbonMultiBlockSplit) split).getAllSplits().get(0);
+    } else {
+      fileSplit = (FileSplit) split;
+    }
+
+    // metadata
+    hadoopConf = context.getConfiguration();
+    if (model == null) {
+      CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
+      model = format.createQueryModel(split, context);
+    }
+    carbonTable = model.getTable();
+    List<CarbonDimension> dimensions =
+        carbonTable.getDimensionByTableName(carbonTable.getTableName());
+    dimensionCount = dimensions.size();
+    List<CarbonMeasure> measures =
+        carbonTable.getMeasureByTableName(carbonTable.getTableName());
+    measureCount = measures.size();
+    List<CarbonColumn> carbonColumnList =
+        carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName());
+    storageColumns = carbonColumnList.toArray(new CarbonColumn[carbonColumnList.size()]);
+    isNoDictColumn = CarbonDataProcessorUtil.getNoDictionaryMapping(storageColumns);
+    directDictionaryGenerators = new DirectDictionaryGenerator[storageColumns.length];
+    for (int i = 0; i < storageColumns.length; i++) {
+      if (storageColumns[i].hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+        directDictionaryGenerators[i] = DirectDictionaryKeyGeneratorFactory
+            .getDirectDictionaryGenerator(storageColumns[i].getDataType());
+      }
+    }
+    measureDataTypes = new DataType[measureCount];
+    for (int i = 0; i < measureCount; i++) {
+      measureDataTypes[i] = storageColumns[dimensionCount + i].getDataType();
+    }
+
+    // decode data
+    allNonNull = new BitSet(storageColumns.length);
+    projection = model.getProjectionColumns();
+
+    isRequired = new boolean[storageColumns.length];
+    boolean[] isFiltlerDimensions = model.getIsFilterDimensions();
+    boolean[] isFiltlerMeasures = model.getIsFilterMeasures();
+    isFilterRequired = new boolean[storageColumns.length];
+    filterMap = new int[storageColumns.length];
+    for (int i = 0; i < storageColumns.length; i++) {
+      if (storageColumns[i].isDimension()) {
+        if (isFiltlerDimensions[storageColumns[i].getOrdinal()]) {
+          isRequired[i] = true;
+          isFilterRequired[i] = true;
+          filterMap[i] = storageColumns[i].getOrdinal();
+        }
+      } else {
+        if (isFiltlerMeasures[storageColumns[i].getOrdinal()]) {
+          isRequired[i] = true;
+          isFilterRequired[i] = true;
+          filterMap[i] = carbonTable.getDimensionOrdinalMax() + storageColumns[i].getOrdinal();
+        }
+      }
+    }
+
+    isProjectionRequired = new boolean[storageColumns.length];
+    projectionMap = new int[storageColumns.length];
+    for (int i = 0; i < storageColumns.length; i++) {
+      for (int j = 0; j < projection.length; j++) {
+        if (storageColumns[i].getColName().equals(projection[j].getColName())) {
+          isRequired[i] = true;
+          isProjectionRequired[i] = true;
+          projectionMap[i] = j;
+          break;
+        }
+      }
+    }
+
+    // initialize filter
+    if (null != model.getFilterExpressionResolverTree()) {
+      initializeFilter();
+    } else if (projection.length == 0) {
+      skipScanData = true;
+    }
+
+  }
+
+  private void initializeFilter() {
+
+    List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
+            carbonTable.getMeasureByTableName(carbonTable.getTableName()));
+    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
+    for (int i = 0; i < dimLensWithComplex.length; i++) {
+      dimLensWithComplex[i] = Integer.MAX_VALUE;
+    }
+
+    int[] dictionaryColumnCardinality =
+        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
+    SegmentProperties segmentProperties =
+        new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality);
+    Map<Integer, GenericQueryType> complexDimensionInfoMap = new HashMap<>();
+
+    FilterResolverIntf resolverIntf = model.getFilterExpressionResolverTree();
+    filter = FilterUtil.getFilterExecuterTree(resolverIntf, segmentProperties,
+        complexDimensionInfoMap);
+    // for row filter, we need update column index
+    FilterUtil.updateIndexOfColumnExpression(resolverIntf.getFilterExpression(),
+        carbonTable.getDimensionOrdinalMax());
+
+  }
+
+  public void setQueryModel(QueryModel model) {
+    this.model = model;
+  }
+
+  private byte[] getSyncMarker(String filePath) throws IOException {
+    CarbonHeaderReader headerReader = new CarbonHeaderReader(filePath);
+    FileHeader header = headerReader.readHeader();
+    return header.getSync_marker();
+  }
+
+  public void setUseRawRow(boolean useRawRow) {
+    this.useRawRow = useRawRow;
+  }
+
+  private void initializeAtFirstRow() throws IOException {
+    filterValues = new Object[carbonTable.getDimensionOrdinalMax() + measureCount];
+    filterRow = new RowImpl();
+    filterRow.setValues(filterValues);
+
+    outputValues = new Object[projection.length];
+    outputRow = new GenericInternalRow(outputValues);
+
+    Path file = fileSplit.getPath();
+
+    byte[] syncMarker = getSyncMarker(file.toString());
+
+    FileSystem fs = file.getFileSystem(hadoopConf);
+
+    int bufferSize = Integer.parseInt(hadoopConf.get(CarbonStreamInputFormat.READ_BUFFER_SIZE,
+        CarbonStreamInputFormat.READ_BUFFER_SIZE_DEFAULT));
+
+    FSDataInputStream fileIn = fs.open(file, bufferSize);
+    fileIn.seek(fileSplit.getStart());
+    input = new StreamBlockletReader(syncMarker, fileIn, fileSplit.getLength(),
+        fileSplit.getStart() == 0);
+
+    cacheProvider = CacheProvider.getInstance();
+    cache = cacheProvider.createCache(CacheType.FORWARD_DICTIONARY);
+    queryTypes = CarbonStreamInputFormat.getComplexDimensions(carbonTable, storageColumns, cache);
+
+    outputSchema = new StructType((StructField[])
+        DataTypeUtil.getDataTypeConverter().convertCarbonSchemaToSparkSchema(projection));
+  }
+
+  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+    if (isFirstRow) {
+      isFirstRow = false;
+      initializeAtFirstRow();
+    }
+    if (isFinished) {
+      return false;
+    }
+
+    if (isVectorReader) {
+      return nextColumnarBatch();
+    }
+
+    return nextRow();
+  }
+
+  /**
+   * for vector reader, check next columnar batch
+   */
+  private boolean nextColumnarBatch() throws IOException {
+    boolean hasNext;
+    boolean scanMore = false;
+    do {
+      // move to the next blocklet
+      hasNext = input.nextBlocklet();
+      if (hasNext) {
+        // read blocklet header
+        BlockletHeader header = input.readBlockletHeader();
+        if (isScanRequired(header)) {
+          scanMore = !scanBlockletAndFillVector(header);
+        } else {
+          input.skipBlockletData(true);
+          scanMore = true;
+        }
+      } else {
+        isFinished = true;
+        scanMore = false;
+      }
+    } while (scanMore);
+    return hasNext;
+  }
+
+  /**
+   * check next Row
+   */
+  private boolean nextRow() throws IOException {
+    // read row one by one
+    try {
+      boolean hasNext;
+      boolean scanMore = false;
+      do {
+        hasNext = input.hasNext();
+        if (hasNext) {
+          if (skipScanData) {
+            input.nextRow();
+            scanMore = false;
+          } else {
+            if (useRawRow) {
+              // read raw row for streaming handoff which does not require decode raw row
+              readRawRowFromStream();
+            } else {
+              readRowFromStream();
+            }
+            if (null != filter) {
+              scanMore = !filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax());
+            } else {
+              scanMore = false;
+            }
+          }
+        } else {
+          if (input.nextBlocklet()) {
+            BlockletHeader header = input.readBlockletHeader();
+            if (isScanRequired(header)) {
+              if (skipScanData) {
+                input.skipBlockletData(false);
+              } else {
+                input.readBlockletData(header);
+              }
+            } else {
+              input.skipBlockletData(true);
+            }
+            scanMore = true;
+          } else {
+            isFinished = true;
+            scanMore = false;
+          }
+        }
+      } while (scanMore);
+      return hasNext;
+    } catch (FilterUnsupportedException e) {
+      throw new IOException("Failed to filter row in detail reader", e);
+    }
+  }
+
+  @Override public Void getCurrentKey() throws IOException, InterruptedException {
+    return null;
+  }
+
+  @Override public Object getCurrentValue() throws IOException, InterruptedException {
+    if (isVectorReader) {
+      int value = columnarBatch.numValidRows();
+      if (inputMetricsStats != null) {
+        inputMetricsStats.incrementRecordRead((long) value);
+      }
+
+      return columnarBatch;
+    }
+
+    if (inputMetricsStats != null) {
+      inputMetricsStats.incrementRecordRead(1L);
+    }
+
+    return outputRow;
+  }
+
+  private boolean isScanRequired(BlockletHeader header) {
+    // TODO require to implement min-max index
+    if (null == filter) {
+      return true;
+    }
+    return true;
+  }
+
+  private boolean scanBlockletAndFillVector(BlockletHeader header) throws IOException {
+    // if filter is null and output projection is empty, use the row number of blocklet header
+    if (skipScanData) {
+      int rowNums = header.getBlocklet_info().getNum_rows();
+      columnarBatch = ColumnarBatch.allocate(outputSchema, MemoryMode.OFF_HEAP, rowNums);
+      columnarBatch.setNumRows(rowNums);
+      input.skipBlockletData(true);
+      return rowNums > 0;
+    }
+
+    input.readBlockletData(header);
+    columnarBatch = ColumnarBatch.allocate(outputSchema, MemoryMode.OFF_HEAP, input.getRowNums());
+    int rowNum = 0;
+    if (null == filter) {
+      while (input.hasNext()) {
+        readRowFromStream();
+        putRowToColumnBatch(rowNum++);
+      }
+    } else {
+      try {
+        while (input.hasNext()) {
+          readRowFromStream();
+          if (filter.applyFilter(filterRow, carbonTable.getDimensionOrdinalMax())) {
+            putRowToColumnBatch(rowNum++);
+          }
+        }
+      } catch (FilterUnsupportedException e) {
+        throw new IOException("Failed to filter row in vector reader", e);
+      }
+    }
+    columnarBatch.setNumRows(rowNum);
+    return rowNum > 0;
+  }
+
+  private void readRowFromStream() {
+    input.nextRow();
+    short nullLen = input.readShort();
+    BitSet nullBitSet = allNonNull;
+    if (nullLen > 0) {
+      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
+    }
+    int colCount = 0;
+    // primitive type dimension
+    for (; colCount < isNoDictColumn.length; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        if (isNoDictColumn[colCount]) {
+          int v = input.readShort();
+          if (isRequired[colCount]) {
+            byte[] b = input.readBytes(v);
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = b;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] =
+                  DataTypeUtil.getDataBasedOnDataTypeForNoDictionaryColumn(b,
+                      storageColumns[colCount].getDataType());
+            }
+          } else {
+            input.skipBytes(v);
+          }
+        } else if (null != directDictionaryGenerators[colCount]) {
+          if (isRequired[colCount]) {
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = input.copy(4);
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] =
+                  directDictionaryGenerators[colCount].getValueFromSurrogate(input.readInt());
+            } else {
+              input.skipBytes(4);
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        } else {
+          if (isRequired[colCount]) {
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = input.copy(4);
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = input.readInt();
+            } else {
+              input.skipBytes(4);
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        }
+      }
+    }
+    // complex type dimension
+    for (; colCount < dimensionCount; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = null;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        short v = input.readShort();
+        if (isRequired[colCount]) {
+          byte[] b = input.readBytes(v);
+          if (isFilterRequired[colCount]) {
+            filterValues[filterMap[colCount]] = b;
+          }
+          if (isProjectionRequired[colCount]) {
+            outputValues[projectionMap[colCount]] = queryTypes[colCount]
+                .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(b));
+          }
+        } else {
+          input.skipBytes(v);
+        }
+      }
+    }
+    // measure
+    DataType dataType;
+    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
+      if (nullBitSet.get(colCount)) {
+        if (isFilterRequired[colCount]) {
+          filterValues[filterMap[colCount]] = null;
+        }
+        if (isProjectionRequired[colCount]) {
+          outputValues[projectionMap[colCount]] = null;
+        }
+      } else {
+        dataType = measureDataTypes[msrCount];
+        if (dataType == DataTypes.BOOLEAN) {
+          if (isRequired[colCount]) {
+            boolean v = input.readBoolean();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(1);
+          }
+        } else if (dataType == DataTypes.SHORT) {
+          if (isRequired[colCount]) {
+            short v = input.readShort();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(2);
+          }
+        } else if (dataType == DataTypes.INT) {
+          if (isRequired[colCount]) {
+            int v = input.readInt();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(4);
+          }
+        } else if (dataType == DataTypes.LONG) {
+          if (isRequired[colCount]) {
+            long v = input.readLong();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(8);
+          }
+        } else if (dataType == DataTypes.DOUBLE) {
+          if (isRequired[colCount]) {
+            double v = input.readDouble();
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] = v;
+            }
+          } else {
+            input.skipBytes(8);
+          }
+        } else if (DataTypes.isDecimal(dataType)) {
+          int len = input.readShort();
+          if (isRequired[colCount]) {
+            BigDecimal v = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
+            if (isFilterRequired[colCount]) {
+              filterValues[filterMap[colCount]] = v;
+            }
+            if (isProjectionRequired[colCount]) {
+              outputValues[projectionMap[colCount]] =
+                  DataTypeUtil.getDataTypeConverter().convertFromBigDecimalToDecimal(v);
+            }
+          } else {
+            input.skipBytes(len);
+          }
+        }
+      }
+    }
+  }
+
+  private void readRawRowFromStream() {
+    input.nextRow();
+    short nullLen = input.readShort();
+    BitSet nullBitSet = allNonNull;
+    if (nullLen > 0) {
+      nullBitSet = BitSet.valueOf(input.readBytes(nullLen));
+    }
+    int colCount = 0;
+    // primitive type dimension
+    for (; colCount < isNoDictColumn.length; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        outputValues[colCount] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY;
+      } else {
+        if (isNoDictColumn[colCount]) {
+          int v = input.readShort();
+          outputValues[colCount] = input.readBytes(v);
+        } else {
+          outputValues[colCount] = input.readInt();
+        }
+      }
+    }
+    // complex type dimension
+    for (; colCount < dimensionCount; colCount++) {
+      if (nullBitSet.get(colCount)) {
+        outputValues[colCount] = null;
+      } else {
+        short v = input.readShort();
+        outputValues[colCount] = input.readBytes(v);
+      }
+    }
+    // measure
+    DataType dataType;
+    for (int msrCount = 0; msrCount < measureCount; msrCount++, colCount++) {
+      if (nullBitSet.get(colCount)) {
+        outputValues[colCount] = null;
+      } else {
+        dataType = measureDataTypes[msrCount];
+        if (dataType == DataTypes.BOOLEAN) {
+          outputValues[colCount] = input.readBoolean();
+        } else if (dataType == DataTypes.SHORT) {
+          outputValues[colCount] = input.readShort();
+        } else if (dataType == DataTypes.INT) {
+          outputValues[colCount] = input.readInt();
+        } else if (dataType == DataTypes.LONG) {
+          outputValues[colCount] = input.readLong();
+        } else if (dataType == DataTypes.DOUBLE) {
+          outputValues[colCount] = input.readDouble();
+        } else if (DataTypes.isDecimal(dataType)) {
+          int len = input.readShort();
+          outputValues[colCount] = DataTypeUtil.byteToBigDecimal(input.readBytes(len));
+        }
+      }
+    }
+  }
+
+  private void putRowToColumnBatch(int rowId) {
+    for (int i = 0; i < projection.length; i++) {
+      Object value = outputValues[i];
+      ColumnVector col = columnarBatch.column(i);
+      org.apache.spark.sql.types.DataType t = col.dataType();
+      if (null == value) {
+        col.putNull(rowId);
+      } else {
+        if (t == org.apache.spark.sql.types.DataTypes.BooleanType) {
+          col.putBoolean(rowId, (boolean)value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.ByteType) {
+          col.putByte(rowId, (byte) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.ShortType) {
+          col.putShort(rowId, (short) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.IntegerType) {
+          col.putInt(rowId, (int) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.LongType) {
+          col.putLong(rowId, (long) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.FloatType) {
+          col.putFloat(rowId, (float) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.DoubleType) {
+          col.putDouble(rowId, (double) value);
+        } else if (t == org.apache.spark.sql.types.DataTypes.StringType) {
+          UTF8String v = (UTF8String) value;
+          col.putByteArray(rowId, v.getBytes());
+        } else if (t instanceof org.apache.spark.sql.types.DecimalType) {
+          DecimalType dt = (DecimalType)t;
+          Decimal d = Decimal.fromDecimal(value);
+          if (dt.precision() <= Decimal.MAX_INT_DIGITS()) {
+            col.putInt(rowId, (int)d.toUnscaledLong());
+          } else if (dt.precision() <= Decimal.MAX_LONG_DIGITS()) {
+            col.putLong(rowId, d.toUnscaledLong());
+          } else {
+            final BigInteger integer = d.toJavaBigDecimal().unscaledValue();
+            byte[] bytes = integer.toByteArray();
+            col.putByteArray(rowId, bytes, 0, bytes.length);
+          }
+        } else if (t instanceof CalendarIntervalType) {
+          CalendarInterval c = (CalendarInterval) value;
+          col.getChildColumn(0).putInt(rowId, c.months);
+          col.getChildColumn(1).putLong(rowId, c.microseconds);
+        } else if (t instanceof org.apache.spark.sql.types.DateType) {
+          col.putInt(rowId, (int) value);
+        } else if (t instanceof org.apache.spark.sql.types.TimestampType) {
+          col.putLong(rowId, (long) value);
+        }
+      }
+    }
+  }
+
+  @Override public float getProgress() throws IOException, InterruptedException {
+    return 0;
+  }
+
+  public void setVectorReader(boolean isVectorReader) {
+    this.isVectorReader = isVectorReader;
+  }
+
+  public void setInputMetricsStats(InputMetricsStats inputMetricsStats) {
+    this.inputMetricsStats = inputMetricsStats;
+  }
+
+  @Override public void close() throws IOException {
+    if (null != input) {
+      input.close();
+    }
+    if (null != columnarBatch) {
+      columnarBatch.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
new file mode 100644
index 0000000..4e555d3
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
@@ -0,0 +1,325 @@
+/*
+ * 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.carbondata.streaming;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.util.CarbonMetadataUtil;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.format.FileHeader;
+import org.apache.carbondata.processing.loading.BadRecordsLogger;
+import org.apache.carbondata.processing.loading.BadRecordsLoggerProvider;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.DataLoadProcessBuilder;
+import org.apache.carbondata.processing.loading.converter.RowConverter;
+import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.loading.parser.RowParser;
+import org.apache.carbondata.processing.loading.parser.impl.RowParserImpl;
+import org.apache.carbondata.processing.store.writer.AbstractFactDataWriter;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskID;
+
+/**
+ * Stream record writer
+ */
+public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CarbonStreamRecordWriter.class.getName());
+
+  // basic info
+  private Configuration hadoopConf;
+  private CarbonLoadModel carbonLoadModel;
+  private CarbonDataLoadConfiguration configuration;
+  private CarbonTable carbonTable;
+  private int maxRowNums;
+  private int maxCacheSize;
+
+  // parser and converter
+  private RowParser rowParser;
+  private BadRecordsLogger badRecordLogger;
+  private RowConverter converter;
+  private CarbonRow currentRow = new CarbonRow(null);
+
+  // encoder
+  private DataField[] dataFields;
+  private BitSet nullBitSet;
+  private boolean[] isNoDictionaryDimensionColumn;
+  private int dimensionWithComplexCount;
+  private int measureCount;
+  private DataType[] measureDataTypes;
+  private StreamBlockletWriter output = null;
+
+  // data write
+  private String segmentDir;
+  private String fileName;
+  private DataOutputStream outputStream;
+  private boolean isFirstRow = true;
+  private boolean hasException = false;
+
+  CarbonStreamRecordWriter(TaskAttemptContext job) throws IOException {
+    initialize(job);
+  }
+
+  public CarbonStreamRecordWriter(TaskAttemptContext job, CarbonLoadModel carbonLoadModel)
+      throws IOException {
+    this.carbonLoadModel = carbonLoadModel;
+    initialize(job);
+  }
+
+  private void initialize(TaskAttemptContext job) throws IOException {
+    // set basic information
+    hadoopConf = job.getConfiguration();
+    if (carbonLoadModel == null) {
+      carbonLoadModel = CarbonStreamOutputFormat.getCarbonLoadModel(hadoopConf);
+      if (carbonLoadModel == null) {
+        throw new IOException(
+            "CarbonStreamRecordWriter require configuration: mapreduce.output.carbon.load.model");
+      }
+    }
+    String segmentId = CarbonStreamOutputFormat.getSegmentId(hadoopConf);
+    carbonLoadModel.setSegmentId(segmentId);
+    carbonTable = carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable();
+    long taskNo = TaskID.forName(hadoopConf.get("mapred.tip.id")).getId();
+    carbonLoadModel.setTaskNo("" + taskNo);
+    configuration = DataLoadProcessBuilder.createConfiguration(carbonLoadModel);
+    maxRowNums = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_STREAM_BLOCKLET_ROW_NUMS,
+        CarbonStreamOutputFormat.CARBON_STREAM_BLOCKLET_ROW_NUMS_DEFAULT) - 1;
+    maxCacheSize = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE,
+        CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE_DEFAULT);
+
+    segmentDir = CarbonTablePath.getSegmentPath(
+        carbonTable.getAbsoluteTableIdentifier().getTablePath(), segmentId);
+    fileName = CarbonTablePath.getCarbonDataFileName(0, taskNo, 0, 0, "0");
+  }
+
+  private void initializeAtFirstRow() throws IOException, InterruptedException {
+
+    // initialize metadata
+    isNoDictionaryDimensionColumn =
+        CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
+    dimensionWithComplexCount = configuration.getDimensionCount();
+    measureCount = configuration.getMeasureCount();
+    dataFields = configuration.getDataFields();
+    measureDataTypes = new DataType[measureCount];
+    for (int i = 0; i < measureCount; i++) {
+      measureDataTypes[i] =
+          dataFields[dimensionWithComplexCount + i].getColumn().getDataType();
+    }
+
+    // initialize parser and converter
+    rowParser = new RowParserImpl(dataFields, configuration);
+    badRecordLogger = BadRecordsLoggerProvider.createBadRecordLogger(configuration);
+    converter = new RowConverterImpl(configuration.getDataFields(), configuration, badRecordLogger);
+    configuration.setCardinalityFinder(converter);
+    converter.initialize();
+
+    // initialize encoder
+    nullBitSet = new BitSet(dataFields.length);
+    int rowBufferSize = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_ENCODER_ROW_BUFFER_SIZE,
+        CarbonStreamOutputFormat.CARBON_ENCODER_ROW_BUFFER_SIZE_DEFAULT);
+    output = new StreamBlockletWriter(maxCacheSize, maxRowNums, rowBufferSize);
+
+    // initialize data writer
+    String filePath = segmentDir + File.separator + fileName;
+    FileFactory.FileType fileType = FileFactory.getFileType(filePath);
+    CarbonFile carbonFile = FileFactory.getCarbonFile(filePath, fileType);
+    if (carbonFile.exists()) {
+      // if the file is existed, use the append api
+      outputStream = FileFactory.getDataOutputStreamUsingAppend(filePath, fileType);
+    } else {
+      // IF the file is not existed, use the create api
+      outputStream = FileFactory.getDataOutputStream(filePath, fileType);
+      writeFileHeader();
+    }
+
+    isFirstRow = false;
+  }
+
+  @Override public void write(Void key, Object value) throws IOException, InterruptedException {
+    if (isFirstRow) {
+      initializeAtFirstRow();
+    }
+
+    // parse and convert row
+    currentRow.setData(rowParser.parseRow((Object[]) value));
+    converter.convert(currentRow);
+
+    // null bit set
+    nullBitSet.clear();
+    for (int i = 0; i < dataFields.length; i++) {
+      if (null == currentRow.getObject(i)) {
+        nullBitSet.set(i);
+      }
+    }
+    output.nextRow();
+    byte[] b = nullBitSet.toByteArray();
+    output.writeShort(b.length);
+    if (b.length > 0) {
+      output.writeBytes(b);
+    }
+    int dimCount = 0;
+    Object columnValue;
+
+    // primitive type dimension
+    for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
+      columnValue = currentRow.getObject(dimCount);
+      if (null != columnValue) {
+        if (isNoDictionaryDimensionColumn[dimCount]) {
+          byte[] col = (byte[]) columnValue;
+          output.writeShort(col.length);
+          output.writeBytes(col);
+        } else {
+          output.writeInt((int) columnValue);
+        }
+      }
+    }
+    // complex type dimension
+    for (; dimCount < dimensionWithComplexCount; dimCount++) {
+      columnValue = currentRow.getObject(dimCount);
+      if (null != columnValue) {
+        byte[] col = (byte[]) columnValue;
+        output.writeShort(col.length);
+        output.writeBytes(col);
+      }
+    }
+    // measure
+    DataType dataType;
+    for (int msrCount = 0; msrCount < measureCount; msrCount++) {
+      columnValue = currentRow.getObject(dimCount + msrCount);
+      if (null != columnValue) {
+        dataType = measureDataTypes[msrCount];
+        if (dataType == DataTypes.BOOLEAN) {
+          output.writeBoolean((boolean) columnValue);
+        } else if (dataType == DataTypes.SHORT) {
+          output.writeShort((short) columnValue);
+        } else if (dataType == DataTypes.INT) {
+          output.writeInt((int) columnValue);
+        } else if (dataType == DataTypes.LONG) {
+          output.writeLong((long) columnValue);
+        } else if (dataType == DataTypes.DOUBLE) {
+          output.writeDouble((double) columnValue);
+        } else if (DataTypes.isDecimal(dataType)) {
+          BigDecimal val = (BigDecimal) columnValue;
+          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
+          output.writeShort(bigDecimalInBytes.length);
+          output.writeBytes(bigDecimalInBytes);
+        } else {
+          String msg =
+              "unsupported data type:" + dataFields[dimCount + msrCount].getColumn().getDataType()
+                  .getName();
+          LOGGER.error(msg);
+          throw new IOException(msg);
+        }
+      }
+    }
+
+    if (output.isFull()) {
+      appendBlockletToDataFile();
+    }
+  }
+
+  private void writeFileHeader() throws IOException {
+    List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
+            carbonTable.getMeasureByTableName(carbonTable.getTableName()));
+    int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
+    for (int i = 0; i < dimLensWithComplex.length; i++) {
+      dimLensWithComplex[i] = Integer.MAX_VALUE;
+    }
+    int[] dictionaryColumnCardinality =
+        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList);
+    List<Integer> cardinality = new ArrayList<>();
+    List<org.apache.carbondata.format.ColumnSchema> columnSchemaList = AbstractFactDataWriter
+        .getColumnSchemaListAndCardinality(cardinality, dictionaryColumnCardinality,
+            wrapperColumnSchemaList);
+    FileHeader fileHeader =
+        CarbonMetadataUtil.getFileHeader(true, columnSchemaList, System.currentTimeMillis());
+    fileHeader.setIs_footer_present(false);
+    fileHeader.setIs_splitable(true);
+    fileHeader.setSync_marker(CarbonStreamOutputFormat.CARBON_SYNC_MARKER);
+    outputStream.write(CarbonUtil.getByteArray(fileHeader));
+  }
+
+  /**
+   * write a blocklet to file
+   */
+  private void appendBlockletToDataFile() throws IOException {
+    if (output.getRowIndex() == -1) {
+      return;
+    }
+    output.apppendBlocklet(outputStream);
+    outputStream.flush();
+    // reset data
+    output.reset();
+  }
+
+  @Override public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+    try {
+      // append remain buffer data
+      if (!hasException && !isFirstRow) {
+        appendBlockletToDataFile();
+        converter.finish();
+      }
+    } finally {
+      // close resource
+      CarbonUtil.closeStreams(outputStream);
+      if (output != null) {
+        output.close();
+      }
+      if (badRecordLogger != null) {
+        badRecordLogger.closeStreams();
+      }
+    }
+  }
+
+  public String getSegmentDir() {
+    return segmentDir;
+  }
+
+  public String getFileName() {
+    return fileName;
+  }
+
+  public void setHasException(boolean hasException) {
+    this.hasException = hasException;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java b/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java
new file mode 100644
index 0000000..43fe6ed
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletReader.java
@@ -0,0 +1,259 @@
+/*
+ * 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.carbondata.streaming;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.carbondata.core.datastore.compression.Compressor;
+import org.apache.carbondata.core.datastore.compression.CompressorFactory;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.format.BlockletHeader;
+
+/**
+ * stream blocklet reader
+ */
+public class StreamBlockletReader {
+
+  private byte[] buffer;
+  private int offset;
+  private final byte[] syncMarker;
+  private final byte[] syncBuffer;
+  private final int syncLen;
+  private long pos = 0;
+  private final InputStream in;
+  private final long limitStart;
+  private final long limitEnd;
+  private boolean isAlreadySync = false;
+  private Compressor compressor = CompressorFactory.getInstance().getCompressor();
+  private int rowNums = 0;
+  private int rowIndex = 0;
+  private boolean isHeaderPresent;
+
+  StreamBlockletReader(byte[] syncMarker, InputStream in, long limit, boolean isHeaderPresent) {
+    this.syncMarker = syncMarker;
+    syncLen = syncMarker.length;
+    syncBuffer = new byte[syncLen];
+    this.in = in;
+    limitStart = limit;
+    limitEnd = limitStart + syncLen;
+    this.isHeaderPresent = isHeaderPresent;
+  }
+
+  private void ensureCapacity(int capacity) {
+    if (buffer == null || capacity > buffer.length) {
+      buffer = new byte[capacity];
+    }
+  }
+
+  /**
+   * find the first position of sync_marker in input stream
+   */
+  private boolean sync() throws IOException {
+    if (!readBytesFromStream(syncBuffer, 0, syncLen)) {
+      return false;
+    }
+    boolean skipHeader = false;
+    for (int i = 0; i < limitStart; i++) {
+      int j = 0;
+      for (; j < syncLen; j++) {
+        if (syncMarker[j] != syncBuffer[(i + j) % syncLen]) break;
+      }
+      if (syncLen == j) {
+        if (isHeaderPresent) {
+          if (skipHeader) {
+            return true;
+          } else {
+            skipHeader = true;
+          }
+        } else {
+          return true;
+        }
+      }
+      int value = in.read();
+      if (-1 == value) {
+        return false;
+      }
+      syncBuffer[i % syncLen] = (byte) value;
+      pos++;
+    }
+    return false;
+  }
+
+  BlockletHeader readBlockletHeader() throws IOException {
+    int len = readIntFromStream();
+    byte[] b = new byte[len];
+    if (!readBytesFromStream(b, 0, len)) {
+      throw new EOFException("Failed to read blocklet header");
+    }
+    BlockletHeader header = CarbonUtil.readBlockletHeader(b);
+    rowNums = header.getBlocklet_info().getNum_rows();
+    rowIndex = 0;
+    return header;
+  }
+
+  void readBlockletData(BlockletHeader header) throws IOException {
+    ensureCapacity(header.getBlocklet_length());
+    offset = 0;
+    int len = readIntFromStream();
+    byte[] b = new byte[len];
+    if (!readBytesFromStream(b, 0, len)) {
+      throw new EOFException("Failed to read blocklet data");
+    }
+    compressor.rawUncompress(b, buffer);
+  }
+
+  void skipBlockletData(boolean reset) throws IOException {
+    int len = readIntFromStream();
+    skip(len);
+    pos += len;
+    if (reset) {
+      this.rowNums = 0;
+      this.rowIndex = 0;
+    }
+  }
+
+  private void skip(int len) throws IOException {
+    long remaining = len;
+    do {
+      long skipLen = in.skip(remaining);
+      remaining -= skipLen;
+    } while (remaining > 0);
+  }
+
+  /**
+   * find the next blocklet
+   */
+  boolean nextBlocklet() throws IOException {
+    if (pos >= limitStart) {
+      return false;
+    }
+    if (isAlreadySync) {
+      if (!readBytesFromStream(syncBuffer, 0, syncLen)) {
+        return false;
+      }
+    } else {
+      isAlreadySync = true;
+      if (!sync()) {
+        return false;
+      }
+    }
+
+    return pos < limitEnd;
+  }
+
+  boolean hasNext() throws IOException {
+    return rowIndex < rowNums;
+  }
+
+  void nextRow() {
+    rowIndex++;
+  }
+
+  int readIntFromStream() throws IOException {
+    int ch1 = in.read();
+    int ch2 = in.read();
+    int ch3 = in.read();
+    int ch4 = in.read();
+    if ((ch1 | ch2 | ch3 | ch4) < 0) throw new EOFException();
+    pos += 4;
+    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
+  }
+
+  /**
+   * Reads <code>len</code> bytes of data from the input stream into
+   * an array of bytes.
+   * @return <code>true</code> if reading data successfully, or
+   * <code>false</code> if there is no more data because the end of the stream has been reached.
+   */
+  boolean readBytesFromStream(byte[] b, int offset, int len) throws IOException {
+    int readLen = in.read(b, offset, len);
+    if (readLen < 0) {
+      return false;
+    }
+    pos += readLen;
+    if (readLen < len) {
+      return readBytesFromStream(b, offset + readLen, len - readLen);
+    } else {
+      return true;
+    }
+  }
+
+  boolean readBoolean() {
+    return (buffer[offset++]) != 0;
+  }
+
+  short readShort() {
+    short v =  (short) ((buffer[offset + 1] & 255) +
+        ((buffer[offset]) << 8));
+    offset += 2;
+    return v;
+  }
+
+  byte[] copy(int len) {
+    byte[] b = new byte[len];
+    System.arraycopy(buffer, offset, b, 0, len);
+    return b;
+  }
+
+  int readInt() {
+    int v = ((buffer[offset + 3] & 255) +
+        ((buffer[offset + 2] & 255) << 8) +
+        ((buffer[offset + 1] & 255) << 16) +
+        ((buffer[offset]) << 24));
+    offset += 4;
+    return v;
+  }
+
+  long readLong() {
+    long v = ((long)(buffer[offset + 7] & 255)) +
+        ((long) (buffer[offset + 6] & 255) << 8) +
+        ((long) (buffer[offset + 5] & 255) << 16) +
+        ((long) (buffer[offset + 4] & 255) << 24) +
+        ((long) (buffer[offset + 3] & 255) << 32) +
+        ((long) (buffer[offset + 2] & 255) << 40) +
+        ((long) (buffer[offset + 1] & 255) << 48) +
+        ((long) (buffer[offset]) << 56);
+    offset += 8;
+    return v;
+  }
+
+  double readDouble() {
+    return Double.longBitsToDouble(readLong());
+  }
+
+  byte[] readBytes(int len) {
+    byte[] b = new byte[len];
+    System.arraycopy(buffer, offset, b, 0, len);
+    offset += len;
+    return b;
+  }
+
+  void skipBytes(int len) {
+    offset += len;
+  }
+
+  int getRowNums() {
+    return rowNums;
+  }
+
+  void close() {
+    CarbonUtil.closeStreams(in);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletWriter.java b/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletWriter.java
new file mode 100644
index 0000000..509e2aa
--- /dev/null
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/StreamBlockletWriter.java
@@ -0,0 +1,152 @@
+/*
+ * 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.carbondata.streaming;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.carbondata.core.datastore.compression.Compressor;
+import org.apache.carbondata.core.datastore.compression.CompressorFactory;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.format.BlockletHeader;
+import org.apache.carbondata.format.BlockletInfo;
+import org.apache.carbondata.format.MutationType;
+
+/**
+ * stream blocklet writer
+ */
+public class StreamBlockletWriter {
+  private byte[] buffer;
+  private int maxSize;
+  private int maxRowNum;
+  private int rowSize;
+  private int count = 0;
+  private int rowIndex = -1;
+  private Compressor compressor = CompressorFactory.getInstance().getCompressor();
+
+  StreamBlockletWriter(int maxSize, int maxRowNum, int rowSize) {
+    buffer = new byte[maxSize];
+    this.maxSize = maxSize;
+    this.maxRowNum = maxRowNum;
+    this.rowSize = rowSize;
+  }
+
+  private void ensureCapacity(int space) {
+    int newcount = space + count;
+    if (newcount > buffer.length) {
+      byte[] newbuf = new byte[Math.max(newcount, buffer.length + rowSize)];
+      System.arraycopy(buffer, 0, newbuf, 0, count);
+      buffer = newbuf;
+    }
+  }
+
+  void reset() {
+    count = 0;
+    rowIndex = -1;
+  }
+
+  byte[] getBytes() {
+    return buffer;
+  }
+
+  int getCount() {
+    return count;
+  }
+
+  int getRowIndex() {
+    return rowIndex;
+  }
+
+  void nextRow() {
+    rowIndex++;
+  }
+
+  boolean isFull() {
+    return rowIndex == maxRowNum || count >= maxSize;
+  }
+
+  void writeBoolean(boolean val) {
+    ensureCapacity(1);
+    buffer[count] = (byte) (val ? 1 : 0);
+    count += 1;
+  }
+
+  void writeShort(int val) {
+    ensureCapacity(2);
+    buffer[count + 1] = (byte) (val);
+    buffer[count] = (byte) (val >>> 8);
+    count += 2;
+  }
+
+  void writeInt(int val) {
+    ensureCapacity(4);
+    buffer[count + 3] = (byte) (val);
+    buffer[count + 2] = (byte) (val >>> 8);
+    buffer[count + 1] = (byte) (val >>> 16);
+    buffer[count] = (byte) (val >>> 24);
+    count += 4;
+  }
+
+  void writeLong(long val) {
+    ensureCapacity(8);
+    buffer[count + 7] = (byte) (val);
+    buffer[count + 6] = (byte) (val >>> 8);
+    buffer[count + 5] = (byte) (val >>> 16);
+    buffer[count + 4] = (byte) (val >>> 24);
+    buffer[count + 3] = (byte) (val >>> 32);
+    buffer[count + 2] = (byte) (val >>> 40);
+    buffer[count + 1] = (byte) (val >>> 48);
+    buffer[count] = (byte) (val >>> 56);
+    count += 8;
+  }
+
+  void writeDouble(double val) {
+    writeLong(Double.doubleToLongBits(val));
+  }
+
+  void writeBytes(byte[] b) {
+    writeBytes(b, 0, b.length);
+  }
+
+  void writeBytes(byte[] b, int off, int len) {
+    ensureCapacity(len);
+    System.arraycopy(b, off, buffer, count, len);
+    count += len;
+  }
+
+  void apppendBlocklet(DataOutputStream outputStream) throws IOException {
+    outputStream.write(CarbonStreamOutputFormat.CARBON_SYNC_MARKER);
+
+    BlockletInfo blockletInfo = new BlockletInfo();
+    blockletInfo.setNum_rows(getRowIndex() + 1);
+    BlockletHeader blockletHeader = new BlockletHeader();
+    blockletHeader.setBlocklet_length(getCount());
+    blockletHeader.setMutation(MutationType.INSERT);
+    blockletHeader.setBlocklet_info(blockletInfo);
+    byte[] headerBytes = CarbonUtil.getByteArray(blockletHeader);
+    outputStream.writeInt(headerBytes.length);
+    outputStream.write(headerBytes);
+
+    byte[] compressed = compressor.compressByte(getBytes(), getCount());
+    outputStream.writeInt(compressed.length);
+    outputStream.write(compressed);
+  }
+
+  void close() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
index 8c9889d..9e83924 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
@@ -42,8 +42,8 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonIndexFileWriter;
 import org.apache.carbondata.format.BlockIndex;
 import org.apache.carbondata.format.BlockletIndex;
-import org.apache.carbondata.hadoop.streaming.CarbonStreamRecordWriter;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.streaming.CarbonStreamRecordWriter;
 
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/scala/org/apache/carbondata/streaming/CarbonSparkStreamingListener.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/CarbonSparkStreamingListener.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/CarbonSparkStreamingListener.scala
deleted file mode 100644
index 6d1fa45..0000000
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/CarbonSparkStreamingListener.scala
+++ /dev/null
@@ -1,31 +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.carbondata.streaming
-
-import org.apache.spark.scheduler.SparkListener
-import org.apache.spark.scheduler.SparkListenerApplicationEnd
-
-class CarbonSparkStreamingListener extends SparkListener {
-
-  /**
-   * When Spark Streaming App stops, remove all locks for stream table.
-   */
-  override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
-    CarbonStreamSparkStreaming.cleanAllLockAfterStop()
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/scala/org/apache/carbondata/streaming/CarbonStreamSparkStreaming.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/CarbonStreamSparkStreaming.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/CarbonStreamSparkStreaming.scala
deleted file mode 100644
index 4aa1517..0000000
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/CarbonStreamSparkStreaming.scala
+++ /dev/null
@@ -1,187 +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.carbondata.streaming
-
-import java.util
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.sql.DataFrame
-import org.apache.spark.sql.execution.streaming.CarbonAppendableStreamSink
-import org.apache.spark.sql.execution.streaming.Sink
-import org.apache.spark.sql.SaveMode
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.streaming.Time
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-
-/**
- * Interface used to write stream data to stream table
- * when integrate with Spark Streaming.
- *
- * NOTE: Current integration with Spark Streaming is an alpha feature.
- */
-class CarbonStreamSparkStreamingWriter(val sparkSession: SparkSession,
-    val carbonTable: CarbonTable,
-    val configuration: Configuration) {
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  private var isInitialize: Boolean = false
-
-  private var lock: ICarbonLock = null
-  private var carbonAppendableStreamSink: Sink = null
-
-  /**
-   * Acquired the lock for stream table
-   */
-  def lockStreamTable(): Unit = {
-    lock = CarbonLockFactory.getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
-      LockUsage.STREAMING_LOCK)
-    if (lock.lockWithRetries()) {
-      LOGGER.info("Acquired the lock for stream table: " +
-                  carbonTable.getDatabaseName + "." +
-                  carbonTable.getTableName)
-    } else {
-      LOGGER.error("Not able to acquire the lock for stream table:" +
-                   carbonTable.getDatabaseName + "." + carbonTable.getTableName)
-      throw new InterruptedException(
-        "Not able to acquire the lock for stream table: " + carbonTable.getDatabaseName + "." +
-        carbonTable.getTableName)
-    }
-  }
-
-  /**
-   * unlock for stream table
-   */
-  def unLockStreamTable(): Unit = {
-    if (null != lock) {
-      lock.unlock()
-      LOGGER.info("unlock for stream table: " +
-                  carbonTable.getDatabaseName + "." +
-                  carbonTable.getTableName)
-    }
-  }
-
-  def initialize(): Unit = {
-    carbonAppendableStreamSink = StreamSinkFactory.createStreamTableSink(
-      sparkSession,
-      configuration,
-      carbonTable,
-      extraOptions.toMap).asInstanceOf[CarbonAppendableStreamSink]
-
-    lockStreamTable()
-
-    isInitialize = true
-  }
-
-  def writeStreamData(dataFrame: DataFrame, time: Time): Unit = {
-    if (!isInitialize) {
-      initialize()
-    }
-    carbonAppendableStreamSink.addBatch(time.milliseconds, dataFrame)
-  }
-
-  private val extraOptions = new scala.collection.mutable.HashMap[String, String]
-  private var mode: SaveMode = SaveMode.ErrorIfExists
-
-  this.option("dbName", carbonTable.getDatabaseName)
-  this.option("tableName", carbonTable.getTableName)
-
-  /**
-   * Specifies the behavior when data or table already exists. Options include:
-   *   - `SaveMode.Overwrite`: overwrite the existing data.
-   *   - `SaveMode.Append`: append the data.
-   *   - `SaveMode.Ignore`: ignore the operation (i.e. no-op).
-   *   - `SaveMode.ErrorIfExists`: default option, throw an exception at runtime.
-   */
-  def mode(saveMode: SaveMode): CarbonStreamSparkStreamingWriter = {
-    if (mode == SaveMode.ErrorIfExists) {
-      mode = saveMode
-    }
-    this
-  }
-
-  /**
-   * Specifies the behavior when data or table already exists. Options include:
-   *   - `overwrite`: overwrite the existing data.
-   *   - `append`: append the data.
-   *   - `ignore`: ignore the operation (i.e. no-op).
-   *   - `error or default`: default option, throw an exception at runtime.
-   */
-  def mode(saveMode: String): CarbonStreamSparkStreamingWriter = {
-    if (mode == SaveMode.ErrorIfExists) {
-      mode = saveMode.toLowerCase(util.Locale.ROOT) match {
-        case "overwrite" => SaveMode.Overwrite
-        case "append" => SaveMode.Append
-        case "ignore" => SaveMode.Ignore
-        case "error" | "default" => SaveMode.ErrorIfExists
-        case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. " +
-          "Accepted save modes are 'overwrite', 'append', 'ignore', 'error'.")
-      }
-    }
-    this
-  }
-
-  /**
-   * Adds an output option
-   */
-  def option(key: String, value: String): CarbonStreamSparkStreamingWriter = {
-    if (!extraOptions.contains(key)) {
-      extraOptions += (key -> value)
-    }
-    this
-  }
-
-  /**
-   * Adds an output option
-   */
-  def option(key: String, value: Boolean): CarbonStreamSparkStreamingWriter =
-    option(key, value.toString)
-
-  /**
-   * Adds an output option
-   */
-  def option(key: String, value: Long): CarbonStreamSparkStreamingWriter =
-    option(key, value.toString)
-
-  /**
-   * Adds an output option
-   */
-  def option(key: String, value: Double): CarbonStreamSparkStreamingWriter =
-    option(key, value.toString)
-}
-
-object CarbonStreamSparkStreaming {
-
-  @transient private val tableMap =
-    new util.HashMap[String, CarbonStreamSparkStreamingWriter]()
-
-  def getTableMap: util.Map[String, CarbonStreamSparkStreamingWriter] = tableMap
-
-  /**
-   * remove all stream lock.
-   */
-  def cleanAllLockAfterStop(): Unit = {
-    tableMap.asScala.values.foreach { writer => writer.unLockStreamTable() }
-    tableMap.clear()
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c723947a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
deleted file mode 100644
index 4df04b9..0000000
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ /dev/null
@@ -1,436 +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.carbondata.streaming
-
-import java.text.SimpleDateFormat
-import java.util
-import java.util.Date
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce.{Job, TaskAttemptID, TaskType}
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
-import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext}
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
-import org.apache.spark.sql.SparkSession
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.datamap.Segment
-import org.apache.carbondata.core.datastore.block.SegmentProperties
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
-import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.scan.result.iterator.RawResultIterator
-import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
-import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonProjection}
-import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
-import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
-import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostStatusUpdateEvent, LoadTablePreStatusUpdateEvent}
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.processing.merger.{CompactionResultSortProcessor, CompactionType}
-import org.apache.carbondata.processing.util.CarbonLoaderUtil
-import org.apache.carbondata.spark.{HandoffResult, HandoffResultImpl}
-import org.apache.carbondata.spark.rdd.CarbonRDD
-import org.apache.carbondata.spark.util.CommonUtil
-
-
-/**
- * partition of the handoff segment
- */
-class HandoffPartition(
-    val rddId: Int,
-    val idx: Int,
-    @transient val inputSplit: CarbonInputSplit
-) extends Partition {
-
-  val split = new SerializableWritable[CarbonInputSplit](inputSplit)
-
-  override val index: Int = idx
-
-  override def hashCode(): Int = 41 * (41 + rddId) + idx
-}
-
-/**
- * package the record reader of the handoff segment to RawResultIterator
- */
-class StreamingRawResultIterator(
-    recordReader: CarbonStreamRecordReader
-) extends RawResultIterator(null, null, null) {
-
-  override def hasNext: Boolean = {
-    recordReader.nextKeyValue()
-  }
-
-  override def next(): Array[Object] = {
-    val rowTmp = recordReader
-      .getCurrentValue
-      .asInstanceOf[GenericInternalRow]
-      .values
-      .asInstanceOf[Array[Object]]
-    val row = new Array[Object](rowTmp.length)
-    System.arraycopy(rowTmp, 0, row, 0, rowTmp.length)
-    row
-  }
-}
-
-/**
- * execute streaming segment handoff
- */
-class StreamHandoffRDD[K, V](
-    sc: SparkContext,
-    result: HandoffResult[K, V],
-    carbonLoadModel: CarbonLoadModel,
-    handOffSegmentId: String
-) extends CarbonRDD[(K, V)](sc, Nil, sc.hadoopConfiguration) {
-
-  private val jobTrackerId: String = {
-    val formatter = new SimpleDateFormat("yyyyMMddHHmm")
-    formatter.format(new Date())
-  }
-
-  override def internalCompute(
-      split: Partition,
-      context: TaskContext
-  ): Iterator[(K, V)] = {
-    carbonLoadModel.setTaskNo("" + split.index)
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    CarbonMetadata.getInstance().addCarbonTable(carbonTable)
-    // the input iterator is using raw row
-    val iteratorList = prepareInputIterator(split, carbonTable)
-
-    CommonUtil.setTempStoreLocation(split.index, carbonLoadModel, true, false)
-    // use CompactionResultSortProcessor to sort data dan write to columnar files
-    val processor = prepareHandoffProcessor(carbonTable)
-    val status = processor.execute(iteratorList)
-
-    new Iterator[(K, V)] {
-      private var finished = false
-
-      override def hasNext: Boolean = {
-        !finished
-      }
-
-      override def next(): (K, V) = {
-        finished = true
-        result.getKey("" + split.index, status)
-      }
-    }
-  }
-
-  /**
-   * prepare input iterator by basing CarbonStreamRecordReader
-   */
-  private def prepareInputIterator(
-      split: Partition,
-      carbonTable: CarbonTable
-  ): util.ArrayList[RawResultIterator] = {
-    val inputSplit = split.asInstanceOf[HandoffPartition].split.value
-    val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0)
-    val hadoopConf = new Configuration()
-    CarbonInputFormat.setDatabaseName(hadoopConf, carbonTable.getDatabaseName)
-    CarbonInputFormat.setTableName(hadoopConf, carbonTable.getTableName)
-    CarbonInputFormat.setTablePath(hadoopConf, carbonTable.getTablePath)
-    val projection = new CarbonProjection
-    val dataFields = carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName)
-    (0 until dataFields.size()).foreach { index =>
-      projection.addColumn(dataFields.get(index).getColName)
-    }
-    CarbonInputFormat.setColumnProjection(hadoopConf, projection)
-    CarbonInputFormat.setTableInfo(hadoopConf, carbonTable.getTableInfo)
-    val attemptContext = new TaskAttemptContextImpl(hadoopConf, attemptId)
-    val format = new CarbonTableInputFormat[Array[Object]]()
-    val model = format.createQueryModel(inputSplit, attemptContext)
-    val inputFormat = new CarbonStreamInputFormat
-    val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
-      .asInstanceOf[CarbonStreamRecordReader]
-    streamReader.setVectorReader(false)
-    streamReader.setQueryModel(model)
-    streamReader.setUseRawRow(true)
-    streamReader.initialize(inputSplit, attemptContext)
-    val iteratorList = new util.ArrayList[RawResultIterator](1)
-    iteratorList.add(new StreamingRawResultIterator(streamReader))
-    iteratorList
-  }
-
-  private def prepareHandoffProcessor(
-      carbonTable: CarbonTable
-  ): CompactionResultSortProcessor = {
-    val wrapperColumnSchemaList = CarbonUtil.getColumnSchemaList(
-      carbonTable.getDimensionByTableName(carbonTable.getTableName),
-      carbonTable.getMeasureByTableName(carbonTable.getTableName))
-    val dimLensWithComplex =
-      (0 until wrapperColumnSchemaList.size()).map(_ => Integer.MAX_VALUE).toArray
-    val dictionaryColumnCardinality =
-      CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchemaList)
-    val segmentProperties =
-      new SegmentProperties(wrapperColumnSchemaList, dictionaryColumnCardinality)
-
-    new CompactionResultSortProcessor(
-      carbonLoadModel,
-      carbonTable,
-      segmentProperties,
-      CompactionType.STREAMING,
-      carbonTable.getTableName,
-      null
-    )
-  }
-
-  /**
-   * get the partitions of the handoff segment
-   */
-  override protected def getPartitions: Array[Partition] = {
-    val job = Job.getInstance(FileFactory.getConfiguration)
-    val inputFormat = new CarbonTableInputFormat[Array[Object]]()
-    val segmentList = new util.ArrayList[Segment](1)
-    segmentList.add(Segment.toSegment(handOffSegmentId))
-    val splits = inputFormat.getSplitsOfStreaming(
-      job,
-      carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getAbsoluteTableIdentifier,
-      segmentList
-    )
-
-    (0 until splits.size()).map { index =>
-      new HandoffPartition(id, index, splits.get(index).asInstanceOf[CarbonInputSplit])
-    }.toArray[Partition]
-  }
-}
-
-object StreamHandoffRDD {
-
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def iterateStreamingHandoff(
-      carbonLoadModel: CarbonLoadModel,
-      sparkSession: SparkSession
-  ): Unit = {
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    val identifier = carbonTable.getAbsoluteTableIdentifier
-    var continueHandoff = false
-    // require handoff lock on table
-    val lock = CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.HANDOFF_LOCK)
-    try {
-      if (lock.lockWithRetries()) {
-        LOGGER.info("Acquired the handoff lock for table" +
-                    s" ${ carbonTable.getDatabaseName }.${ carbonTable.getTableName }")
-        // handoff streaming segment one by one
-        do {
-          val segmentStatusManager = new SegmentStatusManager(identifier)
-          var loadMetadataDetails: Array[LoadMetadataDetails] = null
-          // lock table to read table status file
-          val statusLock = segmentStatusManager.getTableStatusLock
-          try {
-            if (statusLock.lockWithRetries()) {
-              loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-                CarbonTablePath.getMetadataPath(identifier.getTablePath))
-            }
-          } finally {
-            if (null != statusLock) {
-              statusLock.unlock()
-            }
-          }
-          if (null != loadMetadataDetails) {
-            val streamSegments =
-              loadMetadataDetails.filter(_.getSegmentStatus == SegmentStatus.STREAMING_FINISH)
-
-            continueHandoff = streamSegments.length > 0
-            if (continueHandoff) {
-              // handoff a streaming segment
-              val loadMetadataDetail = streamSegments(0)
-              executeStreamingHandoff(
-                carbonLoadModel,
-                sparkSession,
-                loadMetadataDetail.getLoadName
-              )
-            }
-          } else {
-            continueHandoff = false
-          }
-        } while (continueHandoff)
-      }
-    } finally {
-      if (null != lock) {
-        lock.unlock()
-      }
-    }
-  }
-
-  /**
-   * start new thread to execute stream segment handoff
-   */
-  def startStreamingHandoffThread(
-      carbonLoadModel: CarbonLoadModel,
-      sparkSession: SparkSession,
-      isDDL: Boolean
-  ): Unit = {
-    if (isDDL) {
-      iterateStreamingHandoff(carbonLoadModel, sparkSession)
-    } else {
-      // start a new thread to execute streaming segment handoff
-      val handoffThread = new Thread() {
-        override def run(): Unit = {
-          iterateStreamingHandoff(carbonLoadModel, sparkSession)
-        }
-      }
-      handoffThread.start()
-    }
-  }
-
-  /**
-   * invoke StreamHandoffRDD to handoff a streaming segment to a columnar segment
-   */
-  def executeStreamingHandoff(
-      carbonLoadModel: CarbonLoadModel,
-      sparkSession: SparkSession,
-      handoffSegmenId: String
-  ): Unit = {
-    var loadStatus = SegmentStatus.SUCCESS
-    var errorMessage: String = "Handoff failure"
-    try {
-      // generate new columnar segment
-      val newMetaEntry = new LoadMetadataDetails
-      carbonLoadModel.setFactTimeStamp(System.currentTimeMillis())
-      CarbonLoaderUtil.populateNewLoadMetaEntry(
-        newMetaEntry,
-        SegmentStatus.INSERT_IN_PROGRESS,
-        carbonLoadModel.getFactTimeStamp,
-        false)
-      val operationContext = new OperationContext()
-      val loadTablePreStatusUpdateEvent: LoadTablePreStatusUpdateEvent =
-        new LoadTablePreStatusUpdateEvent(
-          carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getCarbonTableIdentifier,
-          carbonLoadModel)
-      OperationListenerBus.getInstance().fireEvent(loadTablePreStatusUpdateEvent, operationContext)
-
-      CarbonLoaderUtil.recordNewLoadMetadata(newMetaEntry, carbonLoadModel, true, false)
-      val loadTablePostStatusUpdateEvent: LoadTablePostStatusUpdateEvent =
-        new LoadTablePostStatusUpdateEvent(carbonLoadModel)
-      OperationListenerBus.getInstance()
-        .fireEvent(loadTablePostStatusUpdateEvent, operationContext)
-      // convert a streaming segment to columnar segment
-      val status = new StreamHandoffRDD(
-        sparkSession.sparkContext,
-        new HandoffResultImpl(),
-        carbonLoadModel,
-        handoffSegmenId).collect()
-
-      status.foreach { x =>
-        if (!x._2) {
-          loadStatus = SegmentStatus.LOAD_FAILURE
-        }
-      }
-    } catch {
-      case ex: Exception =>
-        loadStatus = SegmentStatus.LOAD_FAILURE
-        errorMessage = errorMessage + ": " + ex.getCause.getMessage
-        LOGGER.error(errorMessage)
-        LOGGER.error(ex, s"Handoff failed on streaming segment $handoffSegmenId")
-    }
-
-    if (loadStatus == SegmentStatus.LOAD_FAILURE) {
-      CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel)
-      LOGGER.info("********starting clean up**********")
-      CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
-      LOGGER.info("********clean up done**********")
-      LOGGER.audit(s"Handoff is failed for " +
-                   s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-      LOGGER.error("Cannot write load metadata file as handoff failed")
-      throw new Exception(errorMessage)
-    }
-
-    if (loadStatus == SegmentStatus.SUCCESS) {
-      val done = updateLoadMetadata(handoffSegmenId, carbonLoadModel)
-      if (!done) {
-        val errorMessage = "Handoff failed due to failure in table status updation."
-        LOGGER.audit("Handoff is failed for " +
-                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-        LOGGER.error("Handoff failed due to failure in table status updation.")
-        throw new Exception(errorMessage)
-      }
-      done
-    }
-
-  }
-
-  /**
-   * update streaming segment and new columnar segment
-   */
-  private def updateLoadMetadata(
-      handoffSegmentId: String,
-      loadModel: CarbonLoadModel
-  ): Boolean = {
-    var status = false
-    val metaDataFilepath = loadModel.getCarbonDataLoadSchema.getCarbonTable.getMetadataPath
-    val identifier = loadModel.getCarbonDataLoadSchema.getCarbonTable.getAbsoluteTableIdentifier
-    val metadataPath = CarbonTablePath.getMetadataPath(identifier.getTablePath)
-    val fileType = FileFactory.getFileType(metadataPath)
-    if (!FileFactory.isFileExist(metadataPath, fileType)) {
-      FileFactory.mkdirs(metadataPath, fileType)
-    }
-    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath)
-    val segmentStatusManager = new SegmentStatusManager(identifier)
-    val carbonLock = segmentStatusManager.getTableStatusLock
-    try {
-      if (carbonLock.lockWithRetries()) {
-        LOGGER.info(
-          "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
-          + " for table status updation")
-        val listOfLoadFolderDetailsArray =
-          SegmentStatusManager.readLoadMetadata(metaDataFilepath)
-
-        // update new columnar segment to success status
-        val newSegment =
-          listOfLoadFolderDetailsArray.find(_.getLoadName.equals(loadModel.getSegmentId))
-        if (newSegment.isEmpty) {
-          throw new Exception("Failed to update table status for new segment")
-        } else {
-          newSegment.get.setSegmentStatus(SegmentStatus.SUCCESS)
-          newSegment.get.setLoadEndTime(System.currentTimeMillis())
-        }
-
-        // update streaming segment to compacted status
-        val streamSegment =
-          listOfLoadFolderDetailsArray.find(_.getLoadName.equals(handoffSegmentId))
-        if (streamSegment.isEmpty) {
-          throw new Exception("Failed to update table status for streaming segment")
-        } else {
-          streamSegment.get.setSegmentStatus(SegmentStatus.COMPACTED)
-          streamSegment.get.setMergedLoadName(loadModel.getSegmentId)
-        }
-
-        // refresh table status file
-        SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetailsArray)
-        status = true
-      } else {
-        LOGGER.error("Not able to acquire the lock for Table status updation for table " + loadModel
-          .getDatabaseName() + "." + loadModel.getTableName())
-      }
-    } finally {
-      if (carbonLock.unlock()) {
-        LOGGER.info("Table unlocked successfully after table status updation" +
-                    loadModel.getDatabaseName() + "." + loadModel.getTableName())
-      } else {
-        LOGGER.error("Unable to unlock Table lock for table" + loadModel.getDatabaseName() +
-                     "." + loadModel.getTableName() + " during table status updation")
-      }
-    }
-    status
-  }
-}