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 2016/11/30 17:46:47 UTC

[1/6] incubator-carbondata git commit: add spark2 module

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master d94b99f36 -> 286eccb5e


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
new file mode 100644
index 0000000..c4b5d70
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
@@ -0,0 +1,796 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.optimizer
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.carbon.querystatistics.QueryStatistic
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory
+import org.apache.carbondata.spark.{CarbonAliasDecoderRelation, CarbonFilters}
+
+
+/**
+ * Carbon Optimizer to add dictionary decoder. It does two jobs.
+ * 1. Change the datatype for dictionary encoded column
+ * 2. Add the dictionary decoder operator at appropriate place.
+ */
+class CarbonLateDecodeRule extends Rule[LogicalPlan] with PredicateHelper {
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+  private var relations: Seq[CarbonDecoderRelation] = _
+
+  private def collectCarbonRelation(plan: LogicalPlan): Seq[CarbonDecoderRelation] = {
+    plan collect {
+      case l: LogicalRelation if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        CarbonDecoderRelation(l.attributeMap,
+        l.relation.asInstanceOf[CarbonDatasourceHadoopRelation])
+    }
+  }
+
+  def updateCarbonRelationDataType(plan: LogicalPlan): LogicalPlan = {
+    val relations = plan collect {
+      case l: LogicalRelation if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        l.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+    }
+    if(relations.nonEmpty && !isOptimized(plan)) {
+      val map = mutable.HashMap[ExprId, AttributeReference]()
+      val updateRelationPlan = plan transformDown {
+        case l: LogicalRelation if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+          val relation = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+          val newRelation = updateRelation(relation)
+          val newl = LogicalRelation(newRelation, l.expectedOutputAttributes, l
+              .metastoreTableIdentifier)
+          for(i <- 0 until l.output.size) {
+            map.put(l.output(i).exprId, newl.output(i))
+          }
+          newl
+      }
+
+      updateRelationPlan transformDown {
+        case sort: Sort =>
+          val sortExprs = sort.order.map { s =>
+            s.transform {
+              case attr: AttributeReference =>
+                map.getOrElse(attr.exprId, attr)
+            }.asInstanceOf[SortOrder]
+          }
+          Sort(sortExprs, sort.global, sort.child)
+        case agg: Aggregate =>
+          val aggExps = agg.aggregateExpressions.map { aggExp =>
+            aggExp transform {
+              case attr: AttributeReference =>
+                map.getOrElse(attr.exprId, attr)
+              case other => other
+            }
+          }.asInstanceOf[Seq[NamedExpression]]
+
+          val grpExps = agg.groupingExpressions.map { gexp =>
+            gexp.transform {
+              case attr: AttributeReference =>
+                map.getOrElse(attr.exprId, attr)
+            }
+          }
+          Aggregate(grpExps, aggExps, agg.child)
+        case expand: Expand =>
+          expand.transformExpressions {
+            case attr: AttributeReference =>
+              map.getOrElse(attr.exprId, attr)
+          }
+        case filter: Filter =>
+          val filterExps = filter.condition transform {
+            case attr: AttributeReference =>
+              map.getOrElse(attr.exprId, attr)
+          }
+          Filter(filterExps, filter.child)
+        case p: Project if relations.nonEmpty =>
+          val prExps = p.projectList.map { prExp =>
+            prExp.transform {
+              case attr: AttributeReference =>
+                map.getOrElse(attr.exprId, attr)
+            }
+          }.asInstanceOf[Seq[NamedExpression]]
+          Project(prExps, p.child)
+        case wd: Window if relations.nonEmpty =>
+          val prExps = wd.output.map { prExp =>
+            prExp.transform {
+              case attr: AttributeReference =>
+                map.getOrElse(attr.exprId, attr)
+            }
+          }.asInstanceOf[Seq[Attribute]]
+          val wdExps = wd.windowExpressions.map { gexp =>
+            gexp.transform {
+              case attr: AttributeReference =>
+                map.getOrElse(attr.exprId, attr)
+            }
+          }.asInstanceOf[Seq[NamedExpression]]
+          val partitionSpec = wd.partitionSpec.map{ exp =>
+            exp.transform {
+              case attr: AttributeReference =>
+                map.getOrElse(attr.exprId, attr)
+            }
+          }
+          val orderSpec = wd.orderSpec.map { exp =>
+            exp.transform {
+              case attr: AttributeReference =>
+                map.getOrElse(attr.exprId, attr)
+            }
+          }.asInstanceOf[Seq[SortOrder]]
+          Window(wdExps, partitionSpec, orderSpec, wd.child)
+        case others => others
+      }
+    } else {
+      plan
+    }
+  }
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+    val updatePlan = updateCarbonRelationDataType(plan)
+    relations = collectCarbonRelation(updatePlan)
+    if (relations.nonEmpty && !isOptimized(plan)) {
+      LOGGER.info("Starting to optimize plan")
+      val recorder = CarbonTimeStatisticsFactory.createExecutorRecorder("")
+      val queryStatistic = new QueryStatistic()
+      val result = transformCarbonPlan(updatePlan, relations)
+      queryStatistic.addStatistics("Time taken for Carbon Optimizer to optimize: ",
+        System.currentTimeMillis)
+      recorder.recordStatistics(queryStatistic)
+      recorder.logStatistics()
+      result
+    } else {
+      LOGGER.info("Skip CarbonOptimizer")
+      plan
+    }
+  }
+
+  def isOptimized(plan: LogicalPlan): Boolean = {
+    plan find {
+      case cd: CarbonDictionaryCatalystDecoder => true
+      case other => false
+    } isDefined
+  }
+
+  case class ExtraNodeInfo(var hasCarbonRelation: Boolean)
+
+  def fillNodeInfo(
+      plan: LogicalPlan,
+      extraNodeInfos: java.util.HashMap[LogicalPlan, ExtraNodeInfo]): ExtraNodeInfo = {
+    plan match {
+      case l: LogicalRelation if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        val extraNodeInfo = ExtraNodeInfo(true)
+        extraNodeInfo
+      case others =>
+        val extraNodeInfo = ExtraNodeInfo(false)
+        others.children.foreach { childPlan =>
+          val childExtraNodeInfo = fillNodeInfo(childPlan, extraNodeInfos)
+          if (childExtraNodeInfo.hasCarbonRelation) {
+            extraNodeInfo.hasCarbonRelation = true
+          }
+        }
+        // only put no carbon realtion plan
+        if (!extraNodeInfo.hasCarbonRelation) {
+          extraNodeInfos.put(plan, extraNodeInfo)
+        }
+        extraNodeInfo
+    }
+  }
+
+  /**
+   * Steps for changing the plan.
+   * 1. It finds out the join condition columns and dimension aggregate columns which are need to
+   * be decoded just before that plan executes.
+   * 2. Plan starts transform by adding the decoder to the plan where it needs the decoded data
+   * like dimension aggregate columns decoder under aggregator and join condition decoder under
+   * join children.
+   */
+  def transformCarbonPlan(plan: LogicalPlan,
+      relations: Seq[CarbonDecoderRelation]): LogicalPlan = {
+    if (plan.isInstanceOf[RunnableCommand]) {
+      return plan
+    }
+    var decoder = false
+    val mapOfNonCarbonPlanNodes = new java.util.HashMap[LogicalPlan, ExtraNodeInfo]
+    fillNodeInfo(plan, mapOfNonCarbonPlanNodes)
+    val aliasMap = CarbonAliasDecoderRelation()
+    // collect alias information before hand.
+    collectInformationOnAttributes(plan, aliasMap)
+
+    def hasCarbonRelation(currentPlan: LogicalPlan): Boolean = {
+      val extraNodeInfo = mapOfNonCarbonPlanNodes.get(currentPlan)
+      if (extraNodeInfo == null) {
+        true
+      } else {
+        extraNodeInfo.hasCarbonRelation
+      }
+    }
+
+    val attrMap = new util.HashMap[AttributeReferenceWrapper, CarbonDecoderRelation]()
+    relations.foreach(_.fillAttributeMap(attrMap))
+
+    def addTempDecoder(currentPlan: LogicalPlan): LogicalPlan = {
+      currentPlan match {
+        case sort: Sort if !sort.child.isInstanceOf[CarbonDictionaryTempDecoder] =>
+          val attrsOnSort = new util.HashSet[AttributeReferenceWrapper]()
+          sort.order.map { s =>
+            s.collect {
+              case attr: AttributeReference
+                if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                attrsOnSort.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+            }
+          }
+          var child = sort.child
+          if (attrsOnSort.size() > 0 && !child.isInstanceOf[Sort]) {
+            child = CarbonDictionaryTempDecoder(attrsOnSort,
+              new util.HashSet[AttributeReferenceWrapper](), sort.child)
+          }
+          if (!decoder) {
+            decoder = true
+            CarbonDictionaryTempDecoder(new util.HashSet[AttributeReferenceWrapper](),
+              new util.HashSet[AttributeReferenceWrapper](),
+              Sort(sort.order, sort.global, child),
+              isOuter = true)
+          } else {
+            Sort(sort.order, sort.global, child)
+          }
+
+        case agg: Aggregate if !agg.child.isInstanceOf[CarbonDictionaryTempDecoder] =>
+          val attrsOndimAggs = new util.HashSet[AttributeReferenceWrapper]
+          agg.aggregateExpressions.map {
+            case attr: AttributeReference =>
+            case a@Alias(attr: AttributeReference, name) =>
+            case aggExp: AggregateExpression =>
+              aggExp.transform {
+                case aggExp: AggregateExpression =>
+                  collectDimensionAggregates(aggExp, attrsOndimAggs, aliasMap, attrMap)
+                  aggExp
+              }
+            case others =>
+              others.collect {
+                case attr: AttributeReference
+                  if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                  attrsOndimAggs.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+              }
+          }
+          var child = agg.child
+          // Incase if the child also aggregate then push down decoder to child
+          if (attrsOndimAggs.size() > 0 && !child.equals(agg)) {
+            child = CarbonDictionaryTempDecoder(attrsOndimAggs,
+              new util.HashSet[AttributeReferenceWrapper](),
+              agg.child)
+          }
+          if (!decoder) {
+            decoder = true
+            CarbonDictionaryTempDecoder(new util.HashSet[AttributeReferenceWrapper](),
+              new util.HashSet[AttributeReferenceWrapper](),
+              Aggregate(agg.groupingExpressions, agg.aggregateExpressions, child),
+              isOuter = true)
+          } else {
+            Aggregate(agg.groupingExpressions, agg.aggregateExpressions, child)
+          }
+        case expand: Expand if !expand.child.isInstanceOf[CarbonDictionaryTempDecoder] =>
+          val attrsOnExpand = new util.HashSet[AttributeReferenceWrapper]
+          expand.projections.map {s =>
+            s.map {
+              case attr: AttributeReference =>
+              case a@Alias(attr: AttributeReference, name) =>
+              case others =>
+                others.collect {
+                  case attr: AttributeReference
+                    if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                    attrsOnExpand.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+                }
+            }
+          }
+          var child = expand.child
+          if (attrsOnExpand.size() > 0 && !child.isInstanceOf[Expand]) {
+            child = CarbonDictionaryTempDecoder(attrsOnExpand,
+              new util.HashSet[AttributeReferenceWrapper](),
+              expand.child)
+          }
+          if (!decoder) {
+            decoder = true
+            CarbonDictionaryTempDecoder(new util.HashSet[AttributeReferenceWrapper](),
+              new util.HashSet[AttributeReferenceWrapper](),
+              Expand(expand.projections, expand.output, child),
+              isOuter = true)
+          } else {
+            Expand(expand.projections, expand.output, child)
+          }
+        case filter: Filter if !filter.child.isInstanceOf[CarbonDictionaryTempDecoder] =>
+          val attrsOnConds = new util.HashSet[AttributeReferenceWrapper]
+          // In case the child is join then we cannot push down the filters so decode them earlier
+          if (filter.child.isInstanceOf[Join] || filter.child.isInstanceOf[Sort]) {
+            filter.condition.collect {
+              case attr: AttributeReference =>
+                attrsOnConds.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+            }
+          } else {
+            CarbonFilters
+              .selectFilters(splitConjunctivePredicates(filter.condition), attrsOnConds, aliasMap)
+          }
+
+          var child = filter.child
+          if (attrsOnConds.size() > 0 && !child.isInstanceOf[Filter]) {
+            child = CarbonDictionaryTempDecoder(attrsOnConds,
+              new util.HashSet[AttributeReferenceWrapper](),
+              filter.child)
+          }
+
+          if (!decoder) {
+            decoder = true
+            CarbonDictionaryTempDecoder(new util.HashSet[AttributeReferenceWrapper](),
+              new util.HashSet[AttributeReferenceWrapper](),
+              Filter(filter.condition, child),
+              isOuter = true)
+          } else {
+            Filter(filter.condition, child)
+          }
+
+        case j: Join
+          if !(j.left.isInstanceOf[CarbonDictionaryTempDecoder] ||
+               j.right.isInstanceOf[CarbonDictionaryTempDecoder]) =>
+          val attrsOnJoin = new util.HashSet[Attribute]
+          j.condition match {
+            case Some(expression) =>
+              expression.collect {
+                case attr: AttributeReference
+                  if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                  attrsOnJoin.add(aliasMap.getOrElse(attr, attr))
+              }
+            case _ =>
+          }
+
+          val leftCondAttrs = new util.HashSet[AttributeReferenceWrapper]
+          val rightCondAttrs = new util.HashSet[AttributeReferenceWrapper]
+          if (attrsOnJoin.size() > 0) {
+
+            attrsOnJoin.asScala.map { attr =>
+              if (qualifierPresence(j.left, attr)) {
+                leftCondAttrs.add(AttributeReferenceWrapper(attr))
+              }
+              if (qualifierPresence(j.right, attr)) {
+                rightCondAttrs.add(AttributeReferenceWrapper(attr))
+              }
+            }
+            var leftPlan = j.left
+            var rightPlan = j.right
+            if (leftCondAttrs.size() > 0 &&
+                !leftPlan.isInstanceOf[CarbonDictionaryCatalystDecoder]) {
+              leftPlan = CarbonDictionaryTempDecoder(leftCondAttrs,
+                new util.HashSet[AttributeReferenceWrapper](),
+                j.left)
+            }
+            if (rightCondAttrs.size() > 0 &&
+                !rightPlan.isInstanceOf[CarbonDictionaryCatalystDecoder]) {
+              rightPlan = CarbonDictionaryTempDecoder(rightCondAttrs,
+                new util.HashSet[AttributeReferenceWrapper](),
+                j.right)
+            }
+            if (!decoder) {
+              decoder = true
+              CarbonDictionaryTempDecoder(new util.HashSet[AttributeReferenceWrapper](),
+                new util.HashSet[AttributeReferenceWrapper](),
+                Join(leftPlan, rightPlan, j.joinType, j.condition),
+                isOuter = true)
+            } else {
+              Join(leftPlan, rightPlan, j.joinType, j.condition)
+            }
+          } else {
+            j
+          }
+
+        case p: Project
+          if relations.nonEmpty && !p.child.isInstanceOf[CarbonDictionaryTempDecoder] =>
+          val attrsOnProjects = new util.HashSet[AttributeReferenceWrapper]
+          p.projectList.map {
+            case attr: AttributeReference =>
+            case a@Alias(attr: AttributeReference, name) =>
+            case others =>
+              others.collect {
+                case attr: AttributeReference
+                  if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                  attrsOnProjects.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+              }
+          }
+          var child = p.child
+          if (attrsOnProjects.size() > 0 && !child.isInstanceOf[Project]) {
+            child = CarbonDictionaryTempDecoder(attrsOnProjects,
+              new util.HashSet[AttributeReferenceWrapper](),
+              p.child)
+          }
+          if (!decoder) {
+            decoder = true
+            CarbonDictionaryTempDecoder(new util.HashSet[AttributeReferenceWrapper](),
+              new util.HashSet[AttributeReferenceWrapper](),
+              Project(p.projectList, child),
+              isOuter = true)
+          } else {
+            Project(p.projectList, child)
+          }
+
+        case wd: Window if !wd.child.isInstanceOf[CarbonDictionaryTempDecoder] =>
+          val attrsOnProjects = new util.HashSet[AttributeReferenceWrapper]
+          wd.output.map {
+            case attr: AttributeReference =>
+            case others =>
+              others.collect {
+                case attr: AttributeReference
+                  if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                  attrsOnProjects.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+              }
+          }
+          wd.windowExpressions.map {
+            case others =>
+              others.collect {
+                case attr: AttributeReference
+                  if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                  attrsOnProjects.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+              }
+          }
+          wd.partitionSpec.map{
+            case attr: AttributeReference =>
+            case others =>
+              others.collect {
+                case attr: AttributeReference
+                  if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                  attrsOnProjects.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+              }
+          }
+          wd.orderSpec.map { s =>
+            s.collect {
+              case attr: AttributeReference
+                if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                attrsOnProjects.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+            }
+          }
+          wd.partitionSpec.map { s =>
+            s.collect {
+              case attr: AttributeReference
+                if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+                attrsOnProjects.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+            }
+          }
+          var child = wd.child
+          if (attrsOnProjects.size() > 0 && !child.isInstanceOf[Project]) {
+            child = CarbonDictionaryTempDecoder(attrsOnProjects,
+              new util.HashSet[AttributeReferenceWrapper](),
+              wd.child)
+          }
+          if (!decoder) {
+            decoder = true
+            CarbonDictionaryTempDecoder(new util.HashSet[AttributeReferenceWrapper](),
+              new util.HashSet[AttributeReferenceWrapper](),
+              Window(wd.windowExpressions, wd.partitionSpec, wd.orderSpec, child),
+              isOuter = true)
+          } else {
+            Window(wd.windowExpressions, wd.partitionSpec, wd.orderSpec, child)
+          }
+
+        case l: LogicalRelation if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+          if (!decoder) {
+            decoder = true
+            CarbonDictionaryTempDecoder(new util.HashSet[AttributeReferenceWrapper](),
+              new util.HashSet[AttributeReferenceWrapper](), l, isOuter = true)
+          } else {
+            l
+          }
+
+        case others => others
+      }
+
+    }
+
+    val transFormedPlan =
+      plan transformDown {
+        case cd: CarbonDictionaryTempDecoder if cd.isOuter =>
+          decoder = true
+          cd
+        case currentPlan =>
+          if (hasCarbonRelation(currentPlan)) {
+            addTempDecoder(currentPlan)
+          } else {
+            currentPlan
+          }
+      }
+
+    val processor = new CarbonDecoderProcessor
+    processor.updateDecoders(processor.getDecoderList(transFormedPlan))
+    updateProjection(updateTempDecoder(transFormedPlan, aliasMap, attrMap))
+  }
+
+  private def updateTempDecoder(plan: LogicalPlan,
+      aliasMap: CarbonAliasDecoderRelation,
+      attrMap: java.util.HashMap[AttributeReferenceWrapper, CarbonDecoderRelation]):
+  LogicalPlan = {
+    var allAttrsNotDecode: util.Set[AttributeReferenceWrapper] =
+      new util.HashSet[AttributeReferenceWrapper]()
+    val marker = new CarbonPlanMarker
+    plan transformDown {
+      case cd: CarbonDictionaryTempDecoder if !cd.processed =>
+        cd.processed = true
+        allAttrsNotDecode = cd.attrsNotDecode
+        marker.pushMarker(allAttrsNotDecode)
+        if (cd.isOuter) {
+          CarbonDictionaryCatalystDecoder(relations,
+            ExcludeProfile(cd.getAttrsNotDecode.asScala.toSeq),
+            aliasMap,
+            isOuter = true,
+            cd.child)
+        } else {
+          CarbonDictionaryCatalystDecoder(relations,
+            IncludeProfile(cd.getAttrList.asScala.toSeq),
+            aliasMap,
+            isOuter = false,
+            cd.child)
+        }
+      case cd: CarbonDictionaryCatalystDecoder =>
+        cd
+      case sort: Sort =>
+        val sortExprs = sort.order.map { s =>
+          s.transform {
+            case attr: AttributeReference =>
+              updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+          }.asInstanceOf[SortOrder]
+        }
+        Sort(sortExprs, sort.global, sort.child)
+      case agg: Aggregate if !agg.child.isInstanceOf[CarbonDictionaryCatalystDecoder] =>
+        val aggExps = agg.aggregateExpressions.map { aggExp =>
+          aggExp.transform {
+            case attr: AttributeReference =>
+              updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+          }
+        }.asInstanceOf[Seq[NamedExpression]]
+
+        val grpExps = agg.groupingExpressions.map { gexp =>
+          gexp.transform {
+            case attr: AttributeReference =>
+              updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+          }
+        }
+        Aggregate(grpExps, aggExps, agg.child)
+      case expand: Expand =>
+        expand.transformExpressions {
+          case attr: AttributeReference =>
+            updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+        }
+      case filter: Filter =>
+        val filterExps = filter.condition transform {
+          case attr: AttributeReference =>
+            updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+        }
+        Filter(filterExps, filter.child)
+      case j: Join =>
+        marker.pushBinaryMarker(allAttrsNotDecode)
+        j
+      case u: Union =>
+        marker.pushBinaryMarker(allAttrsNotDecode)
+        u
+      case p: Project if relations.nonEmpty =>
+        val prExps = p.projectList.map { prExp =>
+          prExp.transform {
+            case attr: AttributeReference =>
+              updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+          }
+        }.asInstanceOf[Seq[NamedExpression]]
+        Project(prExps, p.child)
+      case wd: Window if relations.nonEmpty =>
+        val prExps = wd.output.map { prExp =>
+          prExp.transform {
+            case attr: AttributeReference =>
+              updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+          }
+        }.asInstanceOf[Seq[Attribute]]
+        val wdExps = wd.windowExpressions.map { gexp =>
+          gexp.transform {
+            case attr: AttributeReference =>
+              updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+          }
+        }.asInstanceOf[Seq[NamedExpression]]
+        val partitionSpec = wd.partitionSpec.map{ exp =>
+          exp.transform {
+            case attr: AttributeReference =>
+              updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+          }
+        }
+        val orderSpec = wd.orderSpec.map { exp =>
+          exp.transform {
+            case attr: AttributeReference =>
+              updateDataType(attr, attrMap, allAttrsNotDecode, aliasMap)
+          }
+        }.asInstanceOf[Seq[SortOrder]]
+        Window(wdExps, partitionSpec, orderSpec, wd.child)
+
+      case l: LogicalRelation if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        allAttrsNotDecode = marker.revokeJoin()
+        l
+      case others => others
+    }
+  }
+
+  private def updateRelation(relation: CarbonDatasourceHadoopRelation):
+  CarbonDatasourceHadoopRelation = {
+    val fields = relation.schema.fields
+    val numberOfFields = relation.schema.fields.length
+    val newFields = new Array[StructField](numberOfFields)
+    val dictionaryMap = relation.carbonRelation.metaData.dictionaryMap
+    for (i <- 0 until numberOfFields ) {
+      dictionaryMap.get(fields(i).name) match {
+        case Some(true) =>
+          val field = fields(i)
+          newFields(i) = StructField(field.name, IntegerType, field.nullable, field.metadata)
+        case _ => newFields(i) = fields(i)
+      }
+    }
+    CarbonDatasourceHadoopRelation(relation.sparkSession,
+      relation.paths, relation.parameters, Option(StructType(newFields)))
+  }
+
+  private def updateProjection(plan: LogicalPlan): LogicalPlan = {
+    val transFormedPlan = plan transform {
+      case p@Project(projectList: Seq[NamedExpression], cd: CarbonDictionaryCatalystDecoder) =>
+        if (cd.child.isInstanceOf[Filter] || cd.child.isInstanceOf[LogicalRelation]) {
+          Project(projectList: Seq[NamedExpression], cd.child)
+        } else {
+          p
+        }
+      case f@Filter(condition: Expression, cd: CarbonDictionaryCatalystDecoder) =>
+        if (cd.child.isInstanceOf[Project] || cd.child.isInstanceOf[LogicalRelation]) {
+          Filter(condition, cd.child)
+        } else {
+          f
+        }
+    }
+    // Remove unnecessary decoders
+    val finalPlan = transFormedPlan transform {
+      case CarbonDictionaryCatalystDecoder(_, profile, _, false, child)
+        if profile.isInstanceOf[IncludeProfile] && profile.isEmpty =>
+        child
+    }
+    finalPlan
+  }
+
+  private def collectInformationOnAttributes(plan: LogicalPlan,
+      aliasMap: CarbonAliasDecoderRelation) {
+    plan transformAllExpressions  {
+      case a@Alias(exp, name) =>
+        exp match {
+          case attr: Attribute => aliasMap.put(a.toAttribute, attr)
+          case _ => aliasMap.put(a.toAttribute, new AttributeReference("", StringType)())
+        }
+        a
+    }
+    // collect the output of expand and add projections attributes as alias to it.
+    plan.collect {
+      case expand: Expand =>
+        expand.projections.foreach {s =>
+          s.zipWithIndex.foreach { f =>
+            f._1 match {
+              case attr: AttributeReference =>
+                aliasMap.put(expand.output(f._2).toAttribute, attr)
+              case a@Alias(attr: AttributeReference, name) =>
+                aliasMap.put(expand.output(f._2).toAttribute, attr)
+              case others =>
+            }
+          }
+        }
+    }
+  }
+
+  // Collect aggregates on dimensions so that we can add decoder to it.
+  private def collectDimensionAggregates(aggExp: AggregateExpression,
+      attrsOndimAggs: util.HashSet[AttributeReferenceWrapper],
+      aliasMap: CarbonAliasDecoderRelation,
+      attrMap: java.util.HashMap[AttributeReferenceWrapper, CarbonDecoderRelation]) {
+    aggExp collect {
+      case attr: AttributeReference if isDictionaryEncoded(attr, attrMap, aliasMap) =>
+        attrsOndimAggs.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+    }
+  }
+
+  /**
+   * Update the attribute datatype with [IntegerType] if the carbon column is encoded with
+   * dictionary.
+   *
+   */
+  private def updateDataType(attr: Attribute,
+      attrMap: java.util.HashMap[AttributeReferenceWrapper, CarbonDecoderRelation],
+      allAttrsNotDecode: java.util.Set[AttributeReferenceWrapper],
+      aliasMap: CarbonAliasDecoderRelation): Attribute = {
+    val uAttr = aliasMap.getOrElse(attr, attr)
+    val relation = Option(attrMap.get(AttributeReferenceWrapper(uAttr)))
+    if (relation.isDefined) {
+      relation.get.dictionaryMap.get(uAttr.name) match {
+        case Some(true)
+          if !allAttrsNotDecode.contains(AttributeReferenceWrapper(uAttr)) =>
+          val newAttr = AttributeReference(attr.name,
+            IntegerType,
+            attr.nullable,
+            attr.metadata)(attr.exprId)
+          newAttr
+        case _ => attr
+      }
+    } else {
+      attr
+    }
+  }
+
+  private def isDictionaryEncoded(attr: Attribute,
+      attrMap: java.util.HashMap[AttributeReferenceWrapper, CarbonDecoderRelation],
+      aliasMap: CarbonAliasDecoderRelation): Boolean = {
+    val uAttr = aliasMap.getOrElse(attr, attr)
+    val relation = Option(attrMap.get(AttributeReferenceWrapper(uAttr)))
+    if (relation.isDefined) {
+      relation.get.dictionaryMap.get(uAttr.name) match {
+        case Some(true) => true
+        case _ => false
+      }
+    } else {
+      false
+    }
+  }
+
+  def qualifierPresence(plan: LogicalPlan, attr: Attribute): Boolean = {
+    var present = false
+    plan collect {
+      case l: LogicalRelation if l.attributeMap.contains(attr) =>
+        present = true
+    }
+    present
+  }
+}
+
+case class CarbonDecoderRelation(
+    attributeMap: AttributeMap[AttributeReference],
+    carbonRelation: CarbonDatasourceHadoopRelation) {
+
+  val extraAttrs = new ArrayBuffer[Attribute]()
+
+  def addAttribute(attr: Attribute): Unit = {
+    extraAttrs += attr
+  }
+
+  def contains(attr: Attribute): Boolean = {
+    val exists =
+      attributeMap.exists(entry => entry._1.name.equalsIgnoreCase(attr.name) &&
+                                   entry._1.exprId.equals(attr.exprId)) ||
+      extraAttrs.exists(entry => entry.name.equalsIgnoreCase(attr.name) &&
+                                 entry.exprId.equals(attr.exprId))
+    exists
+  }
+
+  def fillAttributeMap(attrMap: java.util.HashMap[AttributeReferenceWrapper,
+    CarbonDecoderRelation]): Unit = {
+    attributeMap.foreach { attr =>
+      attrMap.put(AttributeReferenceWrapper(attr._1), this)
+    }
+  }
+
+  lazy val dictionaryMap = carbonRelation.carbonRelation.metaData.dictionaryMap
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala b/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala
new file mode 100644
index 0000000..f493af1
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/CleanFiles.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.execution.command.{CleanFiles => TableCleanFiles}
+
+/**
+ * clean files api
+ */
+ // scalastyle:off
+object CleanFiles {
+
+  def cleanFiles(spark: SparkSession, dbName: Option[String], tableName: String): Unit = {
+    TableCleanFiles(dbName, tableName).run(spark)
+  }
+
+  def main(args: Array[String]): Unit = {
+
+    if (args.length < 2) {
+      System.err.println("Usage: TableCleanFiles <store path> <table name>");
+      System.exit(1)
+    }
+
+    val storePath = TableAPIUtil.escape(args(0))
+    val (dbName, tableName) = TableAPIUtil.parseSchemaName(TableAPIUtil.escape(args(1)))
+    val spark = TableAPIUtil.spark(storePath, s"TableCleanFiles: $dbName.$tableName")
+    CarbonEnv.init(spark.sqlContext)
+    cleanFiles(spark, Option(dbName), tableName)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala b/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
new file mode 100644
index 0000000..90310d3
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableCompaction, AlterTableModel}
+
+/**
+ * table compaction api
+ */
+ // scalastyle:off
+object Compaction {
+
+  def compaction(spark: SparkSession, dbName: Option[String], tableName: String,
+      compactionType: String): Unit = {
+    AlterTableCompaction(AlterTableModel(dbName, tableName, compactionType, "")).run(spark)
+  }
+
+  def main(args: Array[String]): Unit = {
+    if (args.length < 3) {
+      System.err.println("Usage: TableCompaction <store path> <table name> <major|minor>");
+      System.exit(1)
+    }
+
+    val storePath = TableAPIUtil.escape(args(0))
+    val (dbName, tableName) = TableAPIUtil.parseSchemaName(TableAPIUtil.escape(args(1)))
+    val compactionType = TableAPIUtil.escape(args(2))
+    val spark = TableAPIUtil.spark(storePath, s"TableCompaction: $dbName.$tableName")
+    CarbonEnv.init(spark.sqlContext)
+    compaction(spark, Option(dbName), tableName, compactionType)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala b/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
new file mode 100644
index 0000000..f77a16e
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentByDate.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.execution.command.DeleteLoadsByLoadDate
+
+/**
+ * delete segments before some date
+ */
+// scalastyle:off
+object DeleteSegmentByDate {
+
+  def deleteSegmentByDate(spark: SparkSession, dbName: Option[String], tableName: String,
+      dateValue: String): Unit = {
+    DeleteLoadsByLoadDate(dbName, tableName, "", dateValue).run(spark)
+  }
+
+  def main(args: Array[String]): Unit = {
+    if (args.length < 3) {
+      System.err.println(
+        "Usage: TableDeleteSegmentByDate <store path> <table name> <before date value>");
+      System.exit(1)
+    }
+
+    val storePath = TableAPIUtil.escape(args(0))
+    val (dbName, tableName) = TableAPIUtil.parseSchemaName(TableAPIUtil.escape(args(1)))
+    val dateValue = TableAPIUtil.escape(args(2))
+    val spark = TableAPIUtil.spark(storePath, s"TableCleanFiles: $dbName.$tableName")
+    CarbonEnv.init(spark.sqlContext)
+    deleteSegmentByDate(spark, Option(dbName), tableName, dateValue)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala b/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
new file mode 100644
index 0000000..c3e8626
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/DeleteSegmentById.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.util
+
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.execution.command.DeleteLoadsById
+
+/**
+ * delete segments by id list
+ */
+ // scalastyle:off
+object DeleteSegmentById {
+
+  def extractSegmentIds(segmentIds: String): Seq[String] = {
+    segmentIds.split(",").toSeq
+  }
+
+  def deleteSegmentById(spark: SparkSession, dbName: Option[String], tableName: String,
+      segmentIds: Seq[String]): Unit = {
+    DeleteLoadsById(segmentIds, dbName, tableName).run(spark)
+  }
+
+  def main(args: Array[String]): Unit = {
+
+    if (args.length < 3) {
+      System.err.println(
+        "Usage: TableDeleteSegmentByID <store path> <table name> <segment id list>");
+      System.exit(1)
+    }
+
+    val storePath = TableAPIUtil.escape(args(0))
+    val (dbName, tableName) = TableAPIUtil.parseSchemaName(TableAPIUtil.escape(args(1)))
+    val segmentIds = extractSegmentIds(TableAPIUtil.escape(args(2)))
+    val spark = TableAPIUtil.spark(storePath, s"TableDeleteSegmentById: $dbName.$tableName")
+    CarbonEnv.init(spark.sqlContext)
+    deleteSegmentById(spark, Option(dbName), tableName, segmentIds)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala b/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala
new file mode 100644
index 0000000..5ddffcd
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/ShowSegments.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+
+import org.apache.commons.lang3.StringUtils
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.execution.command.ShowLoads
+import org.apache.spark.sql.types.{StringType, TimestampType}
+
+// scalastyle:off
+object ShowSegments {
+
+  def showSegments(spark: SparkSession, dbName: Option[String], tableName: String,
+      limit: Option[String]): Seq[Row] = {
+    val output =  Seq(AttributeReference("SegmentSequenceId", StringType, nullable = false)(),
+      AttributeReference("Status", StringType, nullable = false)(),
+      AttributeReference("Load Start Time", TimestampType, nullable = false)(),
+      AttributeReference("Load End Time", TimestampType, nullable = false)())
+    ShowLoads(dbName, tableName, limit: Option[String], output).run(spark)
+  }
+
+  def showString(rows: Seq[Row]): String = {
+    val sb = new StringBuilder
+    sb.append("+-----------------+---------------+---------------------+---------------------+\n")
+      .append("|SegmentSequenceId|Status         |Load Start Time      |Load End Time        |\n")
+      .append("+-----------------+---------------+---------------------+---------------------+\n")
+      rows.foreach{row =>
+        sb.append("|")
+          .append(StringUtils.rightPad(row.getString(0), 17))
+          .append("|")
+          .append(StringUtils.rightPad(row.getString(1).substring(0, 15), 15))
+          .append("|")
+          .append(row.getAs[java.sql.Timestamp](2).formatted("yyyy-MM-dd HH:mm:ss.s"))
+          .append("|")
+          .append(row.getAs[java.sql.Timestamp](3).formatted("yyyy-MM-dd HH:mm:ss.s"))
+          .append("|\n")
+      }
+    sb.append("+-----------------+---------------+---------------------+---------------------+\n")
+    sb.toString
+  }
+
+  def parseLimit(limit: String): Int = {
+    Integer.parseInt(limit)
+  }
+
+  def main(args: Array[String]): Unit = {
+
+    if (args.length < 2) {
+      System.err.println("Usage: ShowSegments <store path> <table name> [limit]");
+      System.exit(1)
+    }
+
+    val storePath = TableAPIUtil.escape(args(0))
+    val (dbName, tableName) = TableAPIUtil.parseSchemaName(TableAPIUtil.escape(args(1)))
+
+    val limit = if (args.length >= 3 ) {
+      Some(TableAPIUtil.escape(args(2)))
+    } else {
+      None
+    }
+    val spark = TableAPIUtil.spark(storePath, s"TableCleanFiles: $dbName.$tableName")
+    CarbonEnv.init(spark.sqlContext)
+    val rows = showSegments(spark, Option(dbName), tableName, limit)
+    System.out.println(showString(rows))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala
new file mode 100644
index 0000000..6954981
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
+/**
+ * table api util
+ */
+object TableAPIUtil {
+  def parseSchemaName(tableName: String): (String, String) = {
+    if (tableName.contains(".")) {
+      val parts = tableName.split(".")
+      (parts(0), parts(1))
+    } else {
+      ("default", tableName)
+    }
+  }
+
+  def escape(str: String): String = {
+    val newStr = str.trim
+    if (newStr.startsWith("\"") && newStr.endsWith("\"")) {
+      newStr.substring(1, newStr.length - 1)
+    } else {
+      str
+    }
+  }
+
+  def spark(storePath: String, appName: String): SparkSession = {
+    SparkSession
+        .builder
+        .appName(appName)
+        .master("local")
+        .config(CarbonCommonConstants.STORE_LOCATION, storePath)
+        .getOrCreate()
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
new file mode 100644
index 0000000..8b10aa4
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+
+import java.util.Properties
+
+import scala.collection.{immutable, mutable}
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql._
+import org.apache.spark.sql.execution.command.LoadTable
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * load data api
+ */
+// scalastyle:off
+object TableLoader {
+
+  def extractOptions(propertiesFile: String): immutable.Map[String, String] = {
+    val props = new Properties
+    val path = new Path(propertiesFile)
+    val fs = path.getFileSystem(new Configuration())
+    props.load(fs.open(path))
+    val elments = props.entrySet().iterator()
+    val map = new mutable.HashMap[String, String]()
+    System.out.println("properties file:")
+    while (elments.hasNext) {
+      val elment = elments.next()
+      System.out.println(s"${elment.getKey}=${elment.getValue}")
+      map.put(elment.getKey.asInstanceOf[String], elment.getValue.asInstanceOf[String])
+    }
+
+    immutable.Map(map.toSeq: _*)
+  }
+
+  def extractStorePath(map: immutable.Map[String, String]): String = {
+    map.get(CarbonCommonConstants.STORE_LOCATION) match {
+      case Some(path) => path
+      case None => throw new Exception(s"${CarbonCommonConstants.STORE_LOCATION} can't be empty")
+    }
+  }
+
+  def loadTable(spark: SparkSession, dbName: Option[String], tableName: String, inputPaths: String,
+      options: scala.collection.immutable.Map[String, String]): Unit = {
+    LoadTable(dbName, tableName, inputPaths, Nil, options).run(spark)
+  }
+
+  def main(args: Array[String]): Unit = {
+    if (args.length < 3) {
+      System.err.println("Usage: TableLoader <properties file> <table name> <input files>");
+      System.exit(1)
+    }
+    System.out.println("parameter list:")
+    args.foreach(System.out.println(_))
+    val map = extractOptions(TableAPIUtil.escape(args(0)))
+    val storePath = extractStorePath(map)
+    System.out.println(s"${CarbonCommonConstants.STORE_LOCATION}:$storePath")
+    val (dbName, tableName) = TableAPIUtil.parseSchemaName(TableAPIUtil.escape(args(1)))
+    System.out.println(s"table name: $dbName.$tableName")
+    val inputPaths = TableAPIUtil.escape(args(2))
+
+    val kettleHome = CarbonProperties.getInstance().getProperty("carbon.kettle.home")
+    if (kettleHome == null) {
+      CarbonProperties.getInstance().addProperty("carbon.kettle.home",
+        map.getOrElse("carbon.kettle.home", ""))
+    }
+
+    val spark = TableAPIUtil.spark(storePath, s"TableLoader: $dbName.$tableName")
+
+    CarbonEnv.init(spark.sqlContext)
+    loadTable(spark, Option(dbName), tableName, inputPaths, map)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
new file mode 100644
index 0000000..a635d72
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.carbondata
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.spark.sql.SparkSession
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
+
+class CarbonDataSourceSuite extends FunSuite with BeforeAndAfterAll {
+  var spark: SparkSession = null
+  override def beforeAll(): Unit = {
+    spark = SparkSession
+      .builder()
+      .master("local[4]")
+      .appName("CarbonExample")
+      .enableHiveSupport()
+      .config(CarbonCommonConstants.STORE_LOCATION,
+        s"examples/spark2/target/store")
+      .getOrCreate()
+    spark.sparkContext.setLogLevel("WARN")
+
+    // Drop table
+    spark.sql("DROP TABLE IF EXISTS carbon_table")
+    spark.sql("DROP TABLE IF EXISTS csv_table")
+
+    // Create table
+    spark.sql(
+      s"""
+         | CREATE TABLE carbon_testtable(
+         |    shortField short,
+         |    intField int,
+         |    bigintField long,
+         |    doubleField double,
+         |    stringField string
+         | )
+         | USING org.apache.spark.sql.CarbonSource
+       """.stripMargin)
+  }
+
+  override def afterAll(): Unit = {
+    spark.sql("drop table carbon_testtable")
+    spark.sparkContext.stop()
+    spark = null
+  }
+
+  test("project") {
+    spark.sql("select * from carbon_testtable").collect()
+  }
+
+
+  test("agg") {
+    spark.sql("select stringField, sum(intField) from carbon_testtable group by stringField").collect()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bd51c41..ca28bbf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -101,6 +101,7 @@
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+    <spark.csv.version>1.2.0</spark.csv.version>
     <snappy.version>1.1.2.6</snappy.version>
     <hadoop.version>2.2.0</hadoop.version>
     <kettle.version>4.4.0-stable</kettle.version>
@@ -111,6 +112,22 @@
     <dev.path>${basedir}/dev</dev.path>
   </properties>
 
+  <repositories>
+    <repository>
+      <id>central</id>
+      <!-- This should be at top, it makes maven try the central repo first and then others and hence faster dep resolution -->
+      <name>Maven Repository</name>
+      <url>http://repo1.maven.org/maven2</url>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+    </repository>
+    <repository>
+      <id>pentaho-releases</id>
+      <url>http://repository.pentaho.org/artifactory/repo/</url>
+    </repository>
+  </repositories>
+
   <dependencyManagement>
     <dependencies>
       <dependency>
@@ -168,6 +185,18 @@
         <scope>${scala.deps.scope}</scope>
       </dependency>
       <dependency>
+        <groupId>org.scala-lang</groupId>
+        <artifactId>scala-actors</artifactId>
+        <version>${scala.version}</version>
+        <scope>${scala.deps.scope}</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.scala-lang</groupId>
+        <artifactId>scalap</artifactId>
+        <version>${scala.version}</version>
+        <scope>${scala.deps.scope}</scope>
+      </dependency>
+      <dependency>
         <groupId>org.jmockit</groupId>
         <artifactId>jmockit</artifactId>
         <version>1.10</version>
@@ -280,7 +309,6 @@
     </profile>
     <profile>
       <id>spark-1.5</id>
-      <!-- default -->
       <activation>
         <activeByDefault>true</activeByDefault>
       </activation>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/processing/pom.xml
----------------------------------------------------------------------
diff --git a/processing/pom.xml b/processing/pom.xml
index 7106f3e..c1dd38b 100644
--- a/processing/pom.xml
+++ b/processing/pom.xml
@@ -33,13 +33,6 @@
     <dev.path>${basedir}/../dev</dev.path>
   </properties>
 
-  <repositories>
-    <repository>
-      <id>pentaho-releases</id>
-      <url>http://repository.pentaho.org/artifactory/repo/</url>
-    </repository>
-  </repositories>
-
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java b/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
index 4036a8b..a3bc3a3 100644
--- a/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
+++ b/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
@@ -73,18 +73,19 @@ public class SegmentStatusManager {
   /**
    * This method will return last modified time of tablestatus file
    */
-  public static long getTableStatusLastModifiedTime(AbsoluteTableIdentifier identifier)
-      throws IOException {
+  public static long getTableStatusLastModifiedTime(
+          AbsoluteTableIdentifier identifier) throws IOException {
     String tableStatusPath = CarbonStorePath.getCarbonTablePath(identifier.getStorePath(),
             identifier.getCarbonTableIdentifier()).getTableStatusFilePath();
     if (!FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) {
       return 0L;
     } else {
       return FileFactory.getCarbonFile(tableStatusPath, FileFactory.getFileType(tableStatusPath))
-          .getLastModifiedTime();
+              .getLastModifiedTime();
     }
   }
 
+
   /**
    * get valid segment for given table
    *


[3/6] incubator-carbondata git commit: add spark2 module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
new file mode 100644
index 0000000..2468962
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.execution.command.LoadTable
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.carbon.metadata.datatype.{DataType => CarbonType}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.spark.CarbonOption
+
+class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def saveAsCarbonFile(parameters: Map[String, String] = Map()): Unit = {
+    // create a new table using dataframe's schema and write its content into the table
+    sqlContext.sparkSession.sql(makeCreateTableString(dataFrame.schema,
+    new CarbonOption(parameters)))
+    writeToCarbonFile(parameters)
+  }
+
+  def appendToCarbonFile(parameters: Map[String, String] = Map()): Unit = {
+    writeToCarbonFile(parameters)
+  }
+
+  private def writeToCarbonFile(parameters: Map[String, String] = Map()): Unit = {
+    val options = new CarbonOption(parameters)
+    if (options.tempCSV) {
+      loadTempCSV(options)
+    } else {
+      loadDataFrame(options)
+    }
+  }
+
+  /**
+   * Firstly, saving DataFrame to CSV files
+   * Secondly, load CSV files
+   * @param options
+   * @param sqlContext
+   */
+  private def loadTempCSV(options: CarbonOption): Unit = {
+    // temporary solution: write to csv file, then load the csv into carbon
+    val storePath = CarbonEnv.get.carbonMetastore.storePath
+    val tempCSVFolder = new StringBuilder(storePath).append(CarbonCommonConstants.FILE_SEPARATOR)
+      .append("tempCSV")
+      .append(CarbonCommonConstants.UNDERSCORE).append(options.dbName)
+      .append(CarbonCommonConstants.UNDERSCORE).append(options.tableName)
+      .append(CarbonCommonConstants.UNDERSCORE).append(System.nanoTime()).toString
+    writeToTempCSVFile(tempCSVFolder, options)
+
+    val tempCSVPath = new Path(tempCSVFolder)
+    val fs = tempCSVPath.getFileSystem(dataFrame.sqlContext.sparkContext.hadoopConfiguration)
+
+    def countSize(): Double = {
+      var size: Double = 0
+      val itor = fs.listFiles(tempCSVPath, true)
+      while (itor.hasNext) {
+        val f = itor.next()
+        if (f.getPath.getName.startsWith("part-")) {
+          size += f.getLen
+        }
+      }
+      size
+    }
+
+    LOGGER.info(s"temporary CSV file size: ${countSize / 1024 / 1024} MB")
+
+    try {
+      sqlContext.sql(makeLoadString(tempCSVFolder, options))
+    } finally {
+      fs.delete(tempCSVPath, true)
+    }
+  }
+
+  private def writeToTempCSVFile(tempCSVFolder: String, options: CarbonOption): Unit = {
+    var writer: DataFrameWriter[Row] =
+      dataFrame.write
+        .format(csvPackage)
+        .option("header", "false")
+        .mode(SaveMode.Overwrite)
+
+    if (options.compress) {
+      writer = writer.option("codec", "gzip")
+    }
+
+    writer.save(tempCSVFolder)
+  }
+
+  /**
+   * Loading DataFrame directly without saving DataFrame to CSV files.
+   * @param options
+   */
+  private def loadDataFrame(options: CarbonOption): Unit = {
+    val header = dataFrame.columns.mkString(",")
+    LoadTable(
+      Some(options.dbName),
+      options.tableName,
+      null,
+      Seq(),
+      Map("fileheader" -> header),
+      isOverwriteExist = false,
+      null,
+      Some(dataFrame)).run(sqlContext.sparkSession)
+  }
+
+  private def csvPackage: String = "com.databricks.spark.csv.newapi"
+
+  private def convertToCarbonType(sparkType: DataType): String = {
+    sparkType match {
+      case StringType => CarbonType.STRING.getName
+      case IntegerType => CarbonType.INT.getName
+      case ByteType => CarbonType.INT.getName
+      case ShortType => CarbonType.SHORT.getName
+      case LongType => CarbonType.LONG.getName
+      case FloatType => CarbonType.DOUBLE.getName
+      case DoubleType => CarbonType.DOUBLE.getName
+      case BooleanType => CarbonType.DOUBLE.getName
+      case TimestampType => CarbonType.TIMESTAMP.getName
+      case other => sys.error(s"unsupported type: $other")
+    }
+  }
+
+  private def makeCreateTableString(schema: StructType, options: CarbonOption): String = {
+    val carbonSchema = schema.map { field =>
+      s"${ field.name } ${ convertToCarbonType(field.dataType) }"
+    }
+    s"""
+          CREATE TABLE IF NOT EXISTS ${options.dbName}.${options.tableName}
+          (${ carbonSchema.mkString(", ") })
+          using 'org.apache.spark.sql.CarbonRelationProvider'
+      """
+  }
+
+  private def makeLoadString(csvFolder: String, options: CarbonOption): String = {
+    if (options.useKettle) {
+      s"""
+          LOAD DATA INPATH '$csvFolder'
+          INTO TABLE ${options.dbName}.${options.tableName}
+          OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}')
+      """
+    } else {
+      s"""
+          LOAD DATA INPATH '$csvFolder'
+          INTO TABLE ${options.dbName}.${options.tableName}
+          OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}', 'USE_KETTLE' = 'false')
+      """
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
new file mode 100644
index 0000000..24182ec
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.mapreduce.Job
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
+import org.apache.spark.sql.types.StructType
+
+import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier
+import org.apache.carbondata.hadoop.{CarbonInputFormat, CarbonProjection}
+import org.apache.carbondata.hadoop.util.SchemaReader
+import org.apache.carbondata.scan.expression.Expression
+import org.apache.carbondata.scan.expression.logical.AndExpression
+import org.apache.carbondata.spark.CarbonFilters
+import org.apache.carbondata.spark.merger.TableMeta
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+import org.apache.carbondata.spark.readsupport.SparkRowReadSupportImpl
+import org.apache.carbondata.spark.util.CarbonSparkUtil
+
+case class CarbonDatasourceHadoopRelation(
+    sparkSession: SparkSession,
+    paths: Array[String],
+    parameters: Map[String, String],
+    tableSchema: Option[StructType])
+  extends BaseRelation with PrunedFilteredScan {
+
+  lazy val absIdentifier = AbsoluteTableIdentifier.fromTablePath(paths.head)
+  lazy val carbonTable = SchemaReader.readCarbonTableFromStore(absIdentifier)
+  lazy val carbonRelation: CarbonRelation = {
+    CarbonRelation(
+      carbonTable.getDatabaseName,
+      carbonTable.getFactTableName,
+      CarbonSparkUtil.createSparkMeta(carbonTable),
+      new TableMeta(absIdentifier.getCarbonTableIdentifier, paths.head, carbonTable),
+      None
+    )
+  }
+
+  override def sqlContext: SQLContext = sparkSession.sqlContext
+
+  override def schema: StructType = tableSchema.getOrElse(carbonRelation.schema)
+
+  override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
+    val job = new Job(new JobConf())
+    val conf = new Configuration(job.getConfiguration)
+    val filterExpression: Option[Expression] = filters.flatMap { filter =>
+      CarbonFilters.createCarbonFilter(schema, filter)
+    }.reduceOption(new AndExpression(_, _))
+
+    val projection = new CarbonProjection
+    requiredColumns.foreach(projection.addColumn)
+    CarbonInputFormat.setColumnProjection(conf, projection)
+    CarbonInputFormat.setCarbonReadSupport(classOf[SparkRowReadSupportImpl], conf)
+
+    new CarbonScanRDD[Row](sqlContext.sparkContext, projection, filterExpression.orNull,
+      absIdentifier, carbonTable)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
new file mode 100644
index 0000000..d05aefd
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import org.apache.spark.TaskContext
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.errors.attachTree
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.hive.{CarbonMetastoreTypes, CarbonRelation}
+import org.apache.spark.sql.optimizer.CarbonDecoderRelation
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
+import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
+import org.apache.carbondata.core.carbon.{AbsoluteTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.carbon.querystatistics._
+import org.apache.carbondata.core.util.{CarbonTimeStatisticsFactory, DataTypeUtil}
+import org.apache.carbondata.spark.CarbonAliasDecoderRelation
+
+/**
+ * It decodes the data.
+ *
+ */
+case class CarbonDictionaryDecoder(
+    relations: Seq[CarbonDecoderRelation],
+    profile: CarbonProfile,
+    aliasMap: CarbonAliasDecoderRelation,
+    child: SparkPlan)
+  extends UnaryExecNode {
+
+  override val output: Seq[Attribute] = {
+    child.output.map { a =>
+      val attr = aliasMap.getOrElse(a, a)
+      val relation = relations.find(p => p.contains(attr))
+      if(relation.isDefined && canBeDecoded(attr)) {
+        val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
+        val carbonDimension = carbonTable
+          .getDimensionByName(carbonTable.getFactTableName, attr.name)
+        if (carbonDimension != null &&
+            carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
+            !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          val newAttr = AttributeReference(a.name,
+            convertCarbonToSparkDataType(carbonDimension,
+              relation.get.carbonRelation.carbonRelation),
+            a.nullable,
+            a.metadata)(a.exprId).asInstanceOf[Attribute]
+          newAttr
+        } else {
+          a
+        }
+      } else {
+        a
+      }
+    }
+  }
+
+
+  def canBeDecoded(attr: Attribute): Boolean = {
+    profile match {
+      case ip: IncludeProfile if ip.attributes.nonEmpty =>
+        ip.attributes
+          .exists(a => a.name.equalsIgnoreCase(attr.name) && a.exprId == attr.exprId)
+      case ep: ExcludeProfile =>
+        !ep.attributes
+          .exists(a => a.name.equalsIgnoreCase(attr.name) && a.exprId == attr.exprId)
+      case _ => true
+    }
+  }
+
+  def convertCarbonToSparkDataType(carbonDimension: CarbonDimension,
+      relation: CarbonRelation): types.DataType = {
+    carbonDimension.getDataType match {
+      case DataType.STRING => StringType
+      case DataType.SHORT => ShortType
+      case DataType.INT => IntegerType
+      case DataType.LONG => LongType
+      case DataType.DOUBLE => DoubleType
+      case DataType.BOOLEAN => BooleanType
+      case DataType.DECIMAL =>
+        val scale: Int = carbonDimension.getColumnSchema.getScale
+        val precision: Int = carbonDimension.getColumnSchema.getPrecision
+        if (scale == 0 && precision == 0) {
+          DecimalType(18, 2)
+        } else {
+          DecimalType(precision, scale)
+        }
+      case DataType.TIMESTAMP => TimestampType
+      case DataType.STRUCT =>
+        CarbonMetastoreTypes
+          .toDataType(s"struct<${ relation.getStructChildren(carbonDimension.getColName) }>")
+      case DataType.ARRAY =>
+        CarbonMetastoreTypes
+          .toDataType(s"array<${ relation.getArrayChildren(carbonDimension.getColName) }>")
+    }
+  }
+
+  val getDictionaryColumnIds = {
+    val attributes = child.output
+    val dictIds: Array[(String, ColumnIdentifier, DataType)] = attributes.map { a =>
+      val attr = aliasMap.getOrElse(a, a)
+      val relation = relations.find(p => p.contains(attr))
+      if(relation.isDefined && canBeDecoded(attr)) {
+        val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
+        val carbonDimension =
+          carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
+        if (carbonDimension != null &&
+            carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
+            !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          (carbonTable.getFactTableName, carbonDimension.getColumnIdentifier,
+            carbonDimension.getDataType)
+        } else {
+          (null, null, null)
+        }
+      } else {
+        (null, null, null)
+      }
+
+    }.toArray
+    dictIds
+  }
+
+  override def doExecute(): RDD[InternalRow] = {
+    attachTree(this, "execute") {
+      val storePath = CarbonEnv.get.carbonMetastore.storePath
+      val absoluteTableIdentifiers = relations.map { relation =>
+        val carbonTable = relation.carbonRelation.carbonRelation.metaData.carbonTable
+        (carbonTable.getFactTableName, carbonTable.getAbsoluteTableIdentifier)
+      }.toMap
+
+      if (isRequiredToDecode) {
+        val dataTypes = child.output.map { attr => attr.dataType }
+        child.execute().mapPartitions { iter =>
+          val cacheProvider: CacheProvider = CacheProvider.getInstance
+          val forwardDictionaryCache: Cache[DictionaryColumnUniqueIdentifier, Dictionary] =
+            cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, storePath)
+          val dicts: Seq[Dictionary] = getDictionary(absoluteTableIdentifiers,
+            forwardDictionaryCache)
+          val dictIndex = dicts.zipWithIndex.filter(x => x._1 != null).map(x => x._2)
+          // add a task completion listener to clear dictionary that is a decisive factor for
+          // LRU eviction policy
+          val dictionaryTaskCleaner = TaskContext.get
+          dictionaryTaskCleaner.addTaskCompletionListener(context =>
+            dicts.foreach { dictionary =>
+              if (null != dictionary) {
+                dictionary.clear
+              }
+            }
+          )
+          new Iterator[InternalRow] {
+            val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
+            var flag = true
+            var total = 0L
+            override final def hasNext: Boolean = iter.hasNext
+            override final def next(): InternalRow = {
+              val startTime = System.currentTimeMillis()
+              val row: InternalRow = iter.next()
+              val data = row.toSeq(dataTypes).toArray
+              dictIndex.foreach { index =>
+                if (data(index) != null) {
+                  data(index) = DataTypeUtil.getDataBasedOnDataType(dicts(index)
+                    .getDictionaryValueForKey(data(index).asInstanceOf[Int]),
+                    getDictionaryColumnIds(index)._3)
+                }
+              }
+              val result = unsafeProjection(new GenericMutableRow(data))
+              total += System.currentTimeMillis() - startTime
+              result
+            }
+          }
+        }
+      } else {
+        child.execute()
+      }
+    }
+  }
+
+  private def isRequiredToDecode = {
+    getDictionaryColumnIds.find(p => p._1 != null) match {
+      case Some(value) => true
+      case _ => false
+    }
+  }
+
+  private def getDictionary(atiMap: Map[String, AbsoluteTableIdentifier],
+      cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary]) = {
+    val dicts: Seq[Dictionary] = getDictionaryColumnIds.map { f =>
+      if (f._2 != null) {
+        try {
+          cache.get(new DictionaryColumnUniqueIdentifier(
+            atiMap(f._1).getCarbonTableIdentifier,
+            f._2, f._3))
+        } catch {
+          case _: Throwable => null
+        }
+      } else {
+        null
+      }
+    }
+    dicts
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
new file mode 100644
index 0000000..8028908
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import org.apache.spark.SparkContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.sql.hive.{CarbonMetastore, DistributionUtil}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
+/**
+ * Carbon Environment for unified context
+ */
+case class CarbonEnv(carbonMetastore: CarbonMetastore)
+
+object CarbonEnv extends Logging {
+
+  @volatile private var carbonEnv: CarbonEnv = _
+
+  var initialized = false
+
+  def init(sqlContext: SQLContext): Unit = {
+    if (!initialized) {
+      val catalog = {
+        val storePath = sqlContext.sparkSession.conf.get(
+        CarbonCommonConstants.STORE_LOCATION, "/user/hive/warehouse/carbonstore")
+        new CarbonMetastore(sqlContext.sparkSession.conf, storePath)
+      }
+      carbonEnv = CarbonEnv(catalog)
+      DistributionUtil.numExistingExecutors = sqlContext.sparkContext.schedulerBackend match {
+        case b: CoarseGrainedSchedulerBackend => b.getExecutorIds().length
+        case _ => 0
+      }
+      initialized = true
+    }
+  }
+
+  def get: CarbonEnv = {
+    carbonEnv
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
index cb0b9a5..9e42b44 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonScan.scala
@@ -17,15 +17,16 @@
 
 package org.apache.spark.sql
 
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.hive.CarbonRelation
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.scan.model._
 import org.apache.carbondata.spark.CarbonFilters
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.hive.CarbonRelation
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
 
 case class CarbonScan(
     var attributesRaw: Seq[Attribute],

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
new file mode 100644
index 0000000..fb87ba2
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import java.io.File
+
+import scala.language.implicitConversions
+
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
+import org.apache.spark.sql.execution.CarbonLateDecodeStrategy
+import org.apache.spark.sql.execution.command.{CreateTable, Field}
+import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
+import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.StructType
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.spark.CarbonOption
+
+/**
+ * Carbon relation provider compliant to data source api.
+ * Creates carbon relations
+ */
+class CarbonSource extends CreatableRelationProvider
+    with SchemaRelationProvider with DataSourceRegister {
+
+  override def shortName(): String = "carbondata"
+
+  // called by any write operation like INSERT INTO DDL or DataFrame.write API
+  override def createRelation(
+                               sqlContext: SQLContext,
+                               mode: SaveMode,
+                               parameters: Map[String, String],
+                               data: DataFrame): BaseRelation = {
+    CarbonEnv.init(sqlContext)
+    // User should not specify path since only one store is supported in carbon currently,
+    // after we support multi-store, we can remove this limitation
+    require(!parameters.contains("path"), "'path' should not be specified, " +
+        "the path to store carbon file is the 'storePath' specified when creating CarbonContext")
+
+    val options = new CarbonOption(parameters)
+    val storePath = sqlContext.sparkSession.conf.get(CarbonCommonConstants.STORE_LOCATION)
+    val tablePath = new Path(storePath + "/" + options.dbName + "/" + options.tableName)
+    val isExists = tablePath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
+        .exists(tablePath)
+    val (doSave, doAppend) = (mode, isExists) match {
+      case (SaveMode.ErrorIfExists, true) =>
+        sys.error(s"ErrorIfExists mode, path $storePath already exists.")
+      case (SaveMode.Overwrite, true) =>
+        sqlContext.sparkSession.sql(s"DROP TABLE IF EXISTS ${options.dbName}.${options.tableName}")
+        (true, false)
+      case (SaveMode.Overwrite, false) | (SaveMode.ErrorIfExists, false) =>
+        (true, false)
+      case (SaveMode.Append, _) =>
+        (false, true)
+      case (SaveMode.Ignore, exists) =>
+        (!exists, false)
+    }
+
+    if (doSave) {
+      // save data when the save mode is Overwrite.
+      new CarbonDataFrameWriter(sqlContext, data).saveAsCarbonFile(parameters)
+    } else if (doAppend) {
+      new CarbonDataFrameWriter(sqlContext, data).appendToCarbonFile(parameters)
+    }
+
+    createRelation(sqlContext, parameters, data.schema)
+  }
+
+  // called by DDL operation with a USING clause
+  override def createRelation(
+                               sqlContext: SQLContext,
+                               parameters: Map[String, String],
+                               dataSchema: StructType): BaseRelation = {
+    CarbonEnv.init(sqlContext)
+    addLateDecodeOptimization(sqlContext.sparkSession)
+    val path = createTableIfNotExists(sqlContext.sparkSession, parameters, dataSchema)
+    CarbonDatasourceHadoopRelation(sqlContext.sparkSession, Array(path), parameters,
+      Option(dataSchema))
+
+  }
+
+  private def addLateDecodeOptimization(ss: SparkSession): Unit = {
+    ss.sessionState.experimentalMethods.extraStrategies = Seq(new CarbonLateDecodeStrategy)
+    ss.sessionState.experimentalMethods.extraOptimizations = Seq(new CarbonLateDecodeRule)
+  }
+
+  private def createTableIfNotExists(sparkSession: SparkSession, parameters: Map[String, String],
+                                     dataSchema: StructType): String = {
+    val (dbName, tableName) = parameters.get("path") match {
+      case Some(path) =>
+        val p = path.split(File.separator)
+        ("default", p(p.length - 1))
+      case _ => throw new Exception("do not have dbname and tablename for carbon table")
+    }
+    try {
+      CarbonEnv.get.carbonMetastore.lookupRelation(Option(dbName), tableName)(sparkSession)
+      CarbonEnv.get.carbonMetastore.storePath + s"/$dbName/$tableName"
+    } catch {
+      case ex: NoSuchTableException =>
+        val fields = dataSchema.map { col =>
+          val column = col.name
+          val dataType = Option(col.dataType.toString)
+          val name = Option(col.name)
+          // This is to parse complex data types
+          val x = col.name + ' ' + col.dataType
+          val f: Field = Field(column, dataType, name, None, null)
+          // the data type of the decimal type will be like decimal(10,0)
+          // so checking the start of the string and taking the precision and scale.
+          // resetting the data type with decimal
+          if (f.dataType.getOrElse("").startsWith("decimal")) {
+            val (precision, scale) = TableCreator.getScaleAndPrecision(col.dataType.toString)
+            f.precision = precision
+            f.scale = scale
+            f.dataType = Some("decimal")
+          }
+          f
+        }
+        val map = scala.collection.mutable.Map[String, String]();
+        parameters.foreach { x => map.put(x._1, x._2) }
+        val cm = TableCreator.prepareTableModel(false, Option(dbName), tableName, fields, Nil, map)
+        CreateTable(cm).run(sparkSession)
+        CarbonEnv.get.carbonMetastore.storePath + s"/$dbName/$tableName"
+      case _ => throw new Exception("do not have dbname and tablename for carbon table")
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
new file mode 100644
index 0000000..284af3d
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import java.util.{ArrayList, List}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression, GenericMutableRow}
+
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding
+import org.apache.carbondata.scan.expression.{ColumnExpression, ExpressionResult, UnknownExpression}
+import org.apache.carbondata.scan.expression.conditional.ConditionalExpression
+import org.apache.carbondata.scan.expression.exception.FilterUnsupportedException
+import org.apache.carbondata.scan.filter.intf.{ExpressionType, RowIntf}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+class SparkUnknownExpression(var sparkExp: SparkExpression)
+  extends UnknownExpression with ConditionalExpression {
+
+  private var evaluateExpression: (InternalRow) => Any = sparkExp.eval
+  private var isExecutor: Boolean = false
+  children.addAll(getColumnList())
+
+  override def evaluate(carbonRowInstance: RowIntf): ExpressionResult = {
+
+    val values = carbonRowInstance.getValues.toSeq.map {
+      case s: String => org.apache.spark.unsafe.types.UTF8String.fromString(s)
+      case d: java.math.BigDecimal =>
+        val javaDecVal = new java.math.BigDecimal(d.toString)
+        val scalaDecVal = new scala.math.BigDecimal(javaDecVal)
+        val decConverter = new org.apache.spark.sql.types.Decimal()
+        decConverter.set(scalaDecVal)
+      case value => value
+    }
+    try {
+      val result = evaluateExpression(
+        new GenericMutableRow(values.map(a => a.asInstanceOf[Any]).toArray))
+      val sparkRes = if (isExecutor) {
+        result.asInstanceOf[InternalRow].get(0, sparkExp.dataType)
+      } else {
+        result
+      }
+      new ExpressionResult(CarbonScalaUtil.convertSparkToCarbonDataType(sparkExp.dataType),
+        sparkRes
+      )
+    } catch {
+      case e: Exception => throw new FilterUnsupportedException(e.getMessage)
+    }
+  }
+
+  override def getFilterExpressionType: ExpressionType = {
+    ExpressionType.UNKNOWN
+  }
+
+  override def getString: String = {
+    sparkExp.toString()
+  }
+
+  def setEvaluateExpression(evaluateExpression: (InternalRow) => Any): Unit = {
+    this.evaluateExpression = evaluateExpression
+    isExecutor = true
+  }
+
+  def getColumnList: java.util.List[ColumnExpression] = {
+
+    val lst = new java.util.ArrayList[ColumnExpression]()
+    getColumnListFromExpressionTree(sparkExp, lst)
+    lst
+  }
+  def getLiterals: java.util.List[ExpressionResult] = {
+
+    val lst = new java.util.ArrayList[ExpressionResult]()
+    lst
+  }
+
+  def getAllColumnList: java.util.List[ColumnExpression] = {
+    val lst = new java.util.ArrayList[ColumnExpression]()
+    getAllColumnListFromExpressionTree(sparkExp, lst)
+    lst
+  }
+
+  def isSingleDimension: Boolean = {
+    val lst = new java.util.ArrayList[ColumnExpression]()
+    getAllColumnListFromExpressionTree(sparkExp, lst)
+    if (lst.size == 1 && lst.get(0).isDimension) {
+      true
+    } else {
+      false
+    }
+  }
+
+  def getColumnListFromExpressionTree(sparkCurrentExp: SparkExpression,
+      list: java.util.List[ColumnExpression]): Unit = {
+    sparkCurrentExp.children.foreach(getColumnListFromExpressionTree(_, list))
+  }
+
+
+  def getAllColumnListFromExpressionTree(sparkCurrentExp: SparkExpression,
+      list: List[ColumnExpression]): List[ColumnExpression] = {
+    sparkCurrentExp.children.foreach(getColumnListFromExpressionTree(_, list))
+    list
+  }
+
+  def isDirectDictionaryColumns: Boolean = {
+    val lst = new ArrayList[ColumnExpression]()
+    getAllColumnListFromExpressionTree(sparkExp, lst)
+    if (lst.get(0).getCarbonColumn.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+      true
+    } else {
+      false
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/TableCreator.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/TableCreator.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/TableCreator.scala
new file mode 100644
index 0000000..14decdb
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/TableCreator.scala
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import java.util.regex.{Matcher, Pattern}
+
+import scala.collection.mutable.{LinkedHashSet, Map}
+
+import org.apache.spark.sql.execution.command.{ColumnProperty, Field, PartitionerField, TableModel}
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.DataTypeUtil
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CommonUtil
+
+object TableCreator {
+
+  // detects whether complex dimension is part of dictionary_exclude
+  def isComplexDimDictionaryExclude(dimensionDataType: String): Boolean = {
+    val dimensionType = Array("array", "struct")
+    dimensionType.exists(x => x.equalsIgnoreCase(dimensionDataType))
+  }
+
+  // detects whether double or decimal column is part of dictionary_exclude
+  def isStringAndTimestampColDictionaryExclude(columnDataType: String): Boolean = {
+    val dataTypes = Array("string", "timestamp")
+    dataTypes.exists(x => x.equalsIgnoreCase(columnDataType))
+  }
+
+  // detect dimention data type
+  def isDetectAsDimentionDatatype(dimensionDatatype: String): Boolean = {
+    val dimensionType =
+      Array("string", "stringtype", "array", "arraytype", "struct",
+        "structtype", "timestamp", "timestamptype")
+    dimensionType.exists(x => x.equalsIgnoreCase(dimensionDatatype))
+  }
+
+  protected def extractDimColsAndNoDictionaryFields(fields: Seq[Field],
+                                                    tableProperties: Map[String, String]):
+  (Seq[Field], Seq[String]) = {
+    var dimFields: LinkedHashSet[Field] = LinkedHashSet[Field]()
+    var dictExcludeCols: Array[String] = Array[String]()
+    var noDictionaryDims: Seq[String] = Seq[String]()
+    var dictIncludeCols: Seq[String] = Seq[String]()
+
+    // All excluded cols should be there in create table cols
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).isDefined) {
+      dictExcludeCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
+      dictExcludeCols
+        .map { dictExcludeCol =>
+          if (!fields.exists(x => x.column.equalsIgnoreCase(dictExcludeCol))) {
+            val errormsg = "DICTIONARY_EXCLUDE column: " + dictExcludeCol +
+              " does not exist in table. Please check create table statement."
+            throw new MalformedCarbonCommandException(errormsg)
+          } else {
+            val dataType = fields.find(x =>
+              x.column.equalsIgnoreCase(dictExcludeCol)).get.dataType.get
+            if (isComplexDimDictionaryExclude(dataType)) {
+              val errormsg = "DICTIONARY_EXCLUDE is unsupported for complex datatype column: " +
+                dictExcludeCol
+              throw new MalformedCarbonCommandException(errormsg)
+            } else if (!isStringAndTimestampColDictionaryExclude(dataType)) {
+              val errorMsg = "DICTIONARY_EXCLUDE is unsupported for " + dataType.toLowerCase() +
+                " data type column: " + dictExcludeCol
+              throw new MalformedCarbonCommandException(errorMsg)
+            }
+          }
+        }
+    }
+    // All included cols should be there in create table cols
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).isDefined) {
+      dictIncludeCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).get.split(",").map(_.trim)
+      dictIncludeCols.map { distIncludeCol =>
+        if (!fields.exists(x => x.column.equalsIgnoreCase(distIncludeCol.trim))) {
+          val errormsg = "DICTIONARY_INCLUDE column: " + distIncludeCol.trim +
+            " does not exist in table. Please check create table statement."
+          throw new MalformedCarbonCommandException(errormsg)
+        }
+      }
+    }
+
+    // include cols should contain exclude cols
+    dictExcludeCols.foreach { dicExcludeCol =>
+      if (dictIncludeCols.exists(x => x.equalsIgnoreCase(dicExcludeCol))) {
+        val errormsg = "DICTIONARY_EXCLUDE can not contain the same column: " + dicExcludeCol +
+          " with DICTIONARY_INCLUDE. Please check create table statement."
+        throw new MalformedCarbonCommandException(errormsg)
+      }
+    }
+
+    // by default consider all String cols as dims and if any dictionary exclude is present then
+    // add it to noDictionaryDims list. consider all dictionary excludes/include cols as dims
+    fields.foreach(field => {
+
+      if (dictExcludeCols.toSeq.exists(x => x.equalsIgnoreCase(field.column))) {
+        if (DataTypeUtil.getDataType(field.dataType.get.toUpperCase()) != DataType.TIMESTAMP) {
+          noDictionaryDims :+= field.column
+        }
+        dimFields += field
+      } else if (dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
+        dimFields += (field)
+      } else if (isDetectAsDimentionDatatype(field.dataType.get)) {
+        dimFields += (field)
+      }
+    }
+    )
+
+    (dimFields.toSeq, noDictionaryDims)
+  }
+
+  /**
+   * Extract the Measure Cols fields. By default all non string cols will be measures.
+   *
+   * @param fields
+   * @param tableProperties
+   * @return
+   */
+  protected def extractMsrColsFromFields(fields: Seq[Field],
+                                         tableProperties: Map[String, String]): Seq[Field] = {
+    var msrFields: Seq[Field] = Seq[Field]()
+    var dictIncludedCols: Array[String] = Array[String]()
+    var dictExcludedCols: Array[String] = Array[String]()
+
+    // get all included cols
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).isDefined) {
+      dictIncludedCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).get.split(',').map(_.trim)
+    }
+
+    // get all excluded cols
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).isDefined) {
+      dictExcludedCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
+    }
+
+    // by default consider all non string cols as msrs. consider all include/ exclude cols as dims
+    fields.foreach(field => {
+      if (!isDetectAsDimentionDatatype(field.dataType.get)) {
+        if (!dictIncludedCols.exists(x => x.equalsIgnoreCase(field.column)) &&
+          !dictExcludedCols.exists(x => x.equalsIgnoreCase(field.column))) {
+          msrFields :+= field
+        }
+      }
+    })
+
+    msrFields
+  }
+
+  def getKey(parentColumnName: Option[String],
+             columnName: String): (String, String) = {
+    if (parentColumnName.isDefined) {
+      if (columnName == "val") {
+        (parentColumnName.get, parentColumnName.get + "." + columnName)
+      } else {
+        (parentColumnName.get + "." + columnName, parentColumnName.get + "." + columnName)
+      }
+    } else {
+      (columnName, columnName)
+    }
+  }
+
+  protected def fillColumnProperty(
+      parentColumnName: Option[String],
+      columnName: String,
+      tableProperties: Map[String, String],
+      colPropMap: java.util.HashMap[String, java.util.List[ColumnProperty]]) {
+    val (tblPropKey, colProKey) = getKey(parentColumnName, columnName)
+    val colProps = CommonUtil.getColumnProperties(tblPropKey, tableProperties)
+    if (colProps.isDefined) {
+      colPropMap.put(colProKey, colProps.get)
+    }
+  }
+
+  protected def fillAllChildrenColumnProperty(
+      parent: String,
+      fieldChildren: Option[List[Field]],
+      tableProperties: Map[String, String],
+      colPropMap: java.util.HashMap[String, java.util.List[ColumnProperty]]) {
+    fieldChildren.foreach { fields =>
+      fields.foreach { field =>
+        fillColumnProperty(Some(parent), field.column, tableProperties, colPropMap)
+      }
+    }
+  }
+
+  protected def extractColumnProperties(fields: Seq[Field], tableProperties: Map[String, String]):
+  java.util.Map[String, java.util.List[ColumnProperty]] = {
+    val colPropMap = new java.util.HashMap[String, java.util.List[ColumnProperty]]()
+    fields.foreach { field =>
+      if (field.children.isDefined && field.children.get != null) {
+        fillAllChildrenColumnProperty(field.column, field.children, tableProperties, colPropMap)
+      } else {
+        fillColumnProperty(None, field.column, tableProperties, colPropMap)
+      }
+    }
+    colPropMap
+  }
+
+  def rearrangedColumnGroup(colGroup: String, dims: Seq[Field]): String = {
+    // if columns in column group is not in schema order than arrange it in schema order
+    var colGrpFieldIndx: Seq[Int] = Seq[Int]()
+    colGroup.split(',').map(_.trim).foreach { x =>
+      dims.zipWithIndex.foreach { dim =>
+        if (dim._1.column.equalsIgnoreCase(x)) {
+          colGrpFieldIndx :+= dim._2
+        }
+      }
+    }
+    // sort it
+    colGrpFieldIndx = colGrpFieldIndx.sorted
+    // check if columns in column group is in schema order
+    if (!checkIfInSequence(colGrpFieldIndx)) {
+      throw new MalformedCarbonCommandException("Invalid column group:" + colGroup)
+    }
+    def checkIfInSequence(colGrpFieldIndx: Seq[Int]): Boolean = {
+      for (i <- 0 until (colGrpFieldIndx.length - 1)) {
+        if ((colGrpFieldIndx(i + 1) - colGrpFieldIndx(i)) != 1) {
+          throw new MalformedCarbonCommandException(
+            "Invalid column group,column in group should be contiguous as per schema.")
+        }
+      }
+      true
+    }
+    val colGrpNames: StringBuilder = StringBuilder.newBuilder
+    for (i <- colGrpFieldIndx.indices) {
+      colGrpNames.append(dims(colGrpFieldIndx(i)).column)
+      if (i < (colGrpFieldIndx.length - 1)) {
+        colGrpNames.append(",")
+      }
+    }
+    colGrpNames.toString()
+  }
+
+  /**
+   * Extract the column groups configuration from table properties.
+   * Based on this Row groups of fields will be determined.
+   *
+   * @param tableProperties
+   * @return
+   */
+  protected def updateColumnGroupsInField(tableProperties: Map[String, String],
+                                          noDictionaryDims: Seq[String],
+                                          msrs: Seq[Field],
+                                          dims: Seq[Field]): Seq[String] = {
+    if (tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS).isDefined) {
+
+      var splittedColGrps: Seq[String] = Seq[String]()
+      val nonSplitCols: String = tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS).get
+
+      // row groups will be specified in table properties like -> "(col1,col2),(col3,col4)"
+      // here first splitting the value by () . so that the above will be splitted into 2 strings.
+      // [col1,col2] [col3,col4]
+      val m: Matcher = Pattern.compile("\\(([^)]+)\\)").matcher(nonSplitCols)
+      while (m.find()) {
+        val oneGroup: String = m.group(1)
+        CommonUtil.validateColumnGroup(oneGroup, noDictionaryDims, msrs, splittedColGrps, dims)
+        val arrangedColGrp = rearrangedColumnGroup(oneGroup, dims)
+        splittedColGrps :+= arrangedColGrp
+      }
+      // This will  be furthur handled.
+      CommonUtil.arrangeColGrpsInSchemaOrder(splittedColGrps, dims)
+    } else {
+      null
+    }
+  }
+
+  private def reorderDimensions(dims: Seq[Field]): Seq[Field] = {
+    var complexDimensions: Seq[Field] = Seq()
+    var dimensions: Seq[Field] = Seq()
+    dims.foreach { dimension =>
+      dimension.dataType.getOrElse("NIL") match {
+        case "Array" => complexDimensions = complexDimensions :+ dimension
+        case "Struct" => complexDimensions = complexDimensions :+ dimension
+        case _ => dimensions = dimensions :+ dimension
+      }
+    }
+    dimensions ++ complexDimensions
+  }
+
+  /**
+   * This will extract the no inverted columns fields.
+   * By default all dimensions use inverted index.
+   *
+   * @param fields
+   * @param tableProperties
+   * @return
+   */
+  protected def extractNoInvertedIndexColumns(fields: Seq[Field],
+                                              tableProperties: Map[String, String]):
+  Seq[String] = {
+    // check whether the column name is in fields
+    var noInvertedIdxColsProps: Array[String] = Array[String]()
+    var noInvertedIdxCols: Seq[String] = Seq[String]()
+
+    if (tableProperties.get("NO_INVERTED_INDEX").isDefined) {
+      noInvertedIdxColsProps =
+        tableProperties.get("NO_INVERTED_INDEX").get.split(',').map(_.trim)
+      noInvertedIdxColsProps
+        .map { noInvertedIdxColProp =>
+          if (!fields.exists(x => x.column.equalsIgnoreCase(noInvertedIdxColProp))) {
+            val errormsg = "NO_INVERTED_INDEX column: " + noInvertedIdxColProp +
+              " does not exist in table. Please check create table statement."
+            throw new MalformedCarbonCommandException(errormsg)
+          }
+        }
+    }
+    // check duplicate columns and only 1 col left
+    val distinctCols = noInvertedIdxColsProps.toSet
+    // extract the no inverted index columns
+    fields.foreach(field => {
+      if (distinctCols.exists(x => x.equalsIgnoreCase(field.column))) {
+        noInvertedIdxCols :+= field.column
+      }
+    }
+    )
+    noInvertedIdxCols
+  }
+
+  private def normalizeType(field: Field): Field = {
+    val dataType = field.dataType.getOrElse("NIL")
+    dataType match {
+      case "string" => Field(field.column, Some("String"), field.name, Some(null), field.parent,
+        field.storeType
+      )
+      case "integer" | "int" => Field(field.column, Some("Integer"), field.name, Some(null),
+        field.parent, field.storeType
+      )
+      case "long" => Field(field.column, Some("Long"), field.name, Some(null), field.parent,
+        field.storeType
+      )
+      case "double" => Field(field.column, Some("Double"), field.name, Some(null), field.parent,
+        field.storeType
+      )
+      case "timestamp" => Field(field.column, Some("Timestamp"), field.name, Some(null),
+        field.parent, field.storeType
+      )
+      case "numeric" => Field(field.column, Some("Numeric"), field.name, Some(null), field.parent,
+        field.storeType
+      )
+      case "array" => Field(field.column, Some("Array"), field.name,
+        field.children.map(f => f.map(normalizeType(_))),
+        field.parent, field.storeType
+      )
+      case "struct" => Field(field.column, Some("Struct"), field.name,
+        field.children.map(f => f.map(normalizeType(_))),
+        field.parent, field.storeType
+      )
+      case "bigint" => Field(field.column, Some("BigInt"), field.name, Some(null), field.parent,
+        field.storeType
+      )
+      case "decimal" => Field(field.column, Some("Decimal"), field.name, Some(null), field.parent,
+        field.storeType, field.precision, field.scale
+      )
+      // checking if the nested data type contains the child type as decimal(10,0),
+      // if it is present then extracting the precision and scale. resetting the data type
+      // with Decimal.
+      case _ if (dataType.startsWith("decimal")) =>
+        val (precision, scale) = getScaleAndPrecision(dataType)
+        Field(field.column,
+          Some("Decimal"),
+          field.name,
+          Some(null),
+          field.parent,
+          field.storeType, precision,
+          scale
+        )
+      case _ =>
+        field
+    }
+  }
+
+  private def appendParentForEachChild(field: Field, parentName: String): Field = {
+    field.dataType.getOrElse("NIL") match {
+      case "String" => Field(parentName + "." + field.column, Some("String"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Integer" => Field(parentName + "." + field.column, Some("Integer"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Long" => Field(parentName + "." + field.column, Some("Long"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Double" => Field(parentName + "." + field.column, Some("Double"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Timestamp" => Field(parentName + "." + field.column, Some("Timestamp"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Numeric" => Field(parentName + "." + field.column, Some("Numeric"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Array" => Field(parentName + "." + field.column, Some("Array"),
+        Some(parentName + "." + field.name.getOrElse(None)),
+        field.children
+          .map(f => f.map(appendParentForEachChild(_, parentName + "." + field.column))),
+        parentName)
+      case "Struct" => Field(parentName + "." + field.column, Some("Struct"),
+        Some(parentName + "." + field.name.getOrElse(None)),
+        field.children
+          .map(f => f.map(appendParentForEachChild(_, parentName + "." + field.column))),
+        parentName)
+      case "BigInt" => Field(parentName + "." + field.column, Some("BigInt"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName)
+      case "Decimal" => Field(parentName + "." + field.column, Some("Decimal"),
+        Some(parentName + "." + field.name.getOrElse(None)), Some(null), parentName,
+        field.storeType, field.precision, field.scale)
+      case _ => field
+    }
+  }
+
+  private def addParent(field: Field): Field = {
+    field.dataType.getOrElse("NIL") match {
+      case "Array" => Field(field.column, Some("Array"), field.name,
+        field.children.map(f => f.map(appendParentForEachChild(_, field.column))), field.parent,
+        field.storeType)
+      case "Struct" => Field(field.column, Some("Struct"), field.name,
+        field.children.map(f => f.map(appendParentForEachChild(_, field.column))), field.parent,
+        field.storeType)
+      case _ => field
+    }
+  }
+
+  def getScaleAndPrecision(dataType: String): (Int, Int) = {
+    val m: Matcher = Pattern.compile("^decimal\\(([^)]+)\\)").matcher(dataType)
+    m.find()
+    val matchedString: String = m.group(1)
+    val scaleAndPrecision = matchedString.split(",")
+    (Integer.parseInt(scaleAndPrecision(0).trim), Integer.parseInt(scaleAndPrecision(1).trim))
+  }
+
+  def prepareTableModel(ifNotExistPresent: Boolean, dbName: Option[String]
+                        , tableName: String, fields: Seq[Field],
+                        partitionCols: Seq[PartitionerField],
+                        tableProperties: Map[String, String]): TableModel
+  = {
+
+    val (dims: Seq[Field], noDictionaryDims: Seq[String]) = extractDimColsAndNoDictionaryFields(
+      fields, tableProperties)
+    if (dims.isEmpty) {
+      throw new MalformedCarbonCommandException(s"Table ${
+        dbName.getOrElse(
+          CarbonCommonConstants.DATABASE_DEFAULT_NAME)
+      }.$tableName"
+        +
+        " can not be created without key columns. Please " +
+        "use DICTIONARY_INCLUDE or " +
+        "DICTIONARY_EXCLUDE to set at least one key " +
+        "column " +
+        "if all specified columns are numeric types")
+    }
+    val msrs: Seq[Field] = extractMsrColsFromFields(fields, tableProperties)
+
+    // column properties
+    val colProps = extractColumnProperties(fields, tableProperties)
+    // get column groups configuration from table properties.
+    val groupCols: Seq[String] = updateColumnGroupsInField(tableProperties,
+      noDictionaryDims, msrs, dims)
+
+    // get no inverted index columns from table properties.
+    val noInvertedIdxCols = extractNoInvertedIndexColumns(fields, tableProperties)
+
+    // validate the tableBlockSize from table properties
+    CommonUtil.validateTableBlockSize(tableProperties)
+
+    TableModel(ifNotExistPresent,
+      dbName.getOrElse(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
+      dbName,
+      tableName,
+      tableProperties,
+      reorderDimensions(dims.map(f => normalizeType(f)).map(f => addParent(f))),
+      msrs.map(f => normalizeType(f)),
+      Option(noDictionaryDims),
+      Option(noInvertedIdxCols),
+      groupCols,
+      Some(colProps))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
index c2e3915..4ae8d61 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
@@ -17,114 +17,24 @@
 
 package org.apache.spark.sql.execution
 
-import scala.collection.JavaConverters._
-import org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, Expression, NamedExpression}
-import org.apache.spark.sql._
+import org.apache.spark.sql.{CarbonDictionaryCatalystDecoder, CarbonDictionaryDecoder}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.optimizer.{CarbonAliasDecoderRelation, CarbonDecoderRelation}
-import org.apache.spark.sql.types.IntegerType
 
-///**
-// * Carbon strategy for late decode (convert dictionary key to value as late as possible), which
-// * can improve the aggregation performance and reduce memory usage
-// */
-//private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
-//  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
-//    plan match {
-//      case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
-//        CarbonDictionaryDecoder(relations,
-//          profile,
-//          aliasMap,
-//          planLater(child)
-//        ) :: Nil
-//      case _ => Nil
-//    }
-//  }
-//    /**
-//      * Create carbon scan
-//     */
-//  private def carbonRawScan(projectList: Seq[NamedExpression],
-//      predicates: Seq[Expression],
-//      logicalRelation: LogicalRelation)(sc: SQLContext): SparkPlan = {
-//
-//    val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-//    val tableName: String =
-//      relation.carbonRelation.metaData.carbonTable.getFactTableName.toLowerCase
-//    // Check out any expressions are there in project list. if they are present then we need to
-//    // decode them as well.
-//    val projectSet = AttributeSet(projectList.flatMap(_.references))
-//    val scan = CarbonScan(projectSet.toSeq, relation.carbonRelation, predicates)
-//    projectList.map {
-//      case attr: AttributeReference =>
-//      case Alias(attr: AttributeReference, _) =>
-//      case others =>
-//        others.references.map{f =>
-//          val dictionary = relation.carbonRelation.metaData.dictionaryMap.get(f.name)
-//          if (dictionary.isDefined && dictionary.get) {
-//            scan.attributesNeedToDecode.add(f.asInstanceOf[AttributeReference])
-//          }
-//        }
-//    }
-//    if (scan.attributesNeedToDecode.size() > 0) {
-//      val decoder = getCarbonDecoder(logicalRelation,
-//        sc,
-//        tableName,
-//        scan.attributesNeedToDecode.asScala.toSeq,
-//        scan)
-//      if (scan.unprocessedExprs.nonEmpty) {
-//        val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
-//        ProjectExec(projectList, filterCondToAdd.map(FilterExec(_, decoder)).getOrElse(decoder))
-//      } else {
-//        ProjectExec(projectList, decoder)
-//      }
-//    } else {
-//      ProjectExec(projectList, scan)
-//    }
-//  }
-//
-//  def getCarbonDecoder(logicalRelation: LogicalRelation,
-//      sc: SQLContext,
-//      tableName: String,
-//      projectExprsNeedToDecode: Seq[Attribute],
-//      scan: CarbonScan): CarbonDictionaryDecoder = {
-//    val relation = CarbonDecoderRelation(logicalRelation.attributeMap,
-//      logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation])
-//    val attrs = projectExprsNeedToDecode.map { attr =>
-//      val newAttr = AttributeReference(attr.name,
-//        attr.dataType,
-//        attr.nullable,
-//        attr.metadata)(attr.exprId)
-//      relation.addAttribute(newAttr)
-//      newAttr
-//    }
-//    CarbonDictionaryDecoder(Seq(relation), IncludeProfile(attrs),
-//      CarbonAliasDecoderRelation(), scan)(sc)
-//  }
-//
-//  def isDictionaryEncoded(projectExprsNeedToDecode: Seq[Attribute],
-//      relation: CarbonDatasourceHadoopRelation): Boolean = {
-//    var isEncoded = false
-//    projectExprsNeedToDecode.foreach { attr =>
-//      if (relation.carbonRelation.metaData.dictionaryMap.get(attr.name).getOrElse(false)) {
-//        isEncoded = true
-//      }
-//    }
-//    isEncoded
-//  }
-//
-//  def updateDataType(attr: AttributeReference,
-//      relation: CarbonDatasourceHadoopRelation,
-//      allAttrsNotDecode: java.util.Set[Attribute]): AttributeReference = {
-//    if (relation.carbonRelation.metaData.dictionaryMap.get(attr.name).getOrElse(false) &&
-//        !allAttrsNotDecode.asScala.exists(p => p.name.equals(attr.name))) {
-//      AttributeReference(attr.name,
-//        IntegerType,
-//        attr.nullable,
-//        attr.metadata)(attr.exprId, attr.qualifiers)
-//    } else {
-//      attr
-//    }
-//  }
-//}
+/**
+ * Carbon strategy for late decode (convert dictionary key to value as late as possible), which
+ * can improve the aggregation performance and reduce memory usage
+ */
+private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
+  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+    plan match {
+      case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
+        CarbonDictionaryDecoder(relations,
+          profile,
+          aliasMap,
+          planLater(child)
+        ) :: Nil
+      case _ => Nil
+    }
+  }
+
+}


[4/6] incubator-carbondata git commit: add spark2 module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
new file mode 100644
index 0000000..cfae186
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -0,0 +1,1115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+import java.util.concurrent._
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+import scala.util.Random
+import scala.util.control.Breaks._
+
+import com.databricks.spark.csv.newapi.CarbonTextFile
+import org.apache.hadoop.conf.{Configurable, Configuration}
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.Job
+import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
+import org.apache.spark.{SparkContext, SparkEnv, SparkException}
+import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer}
+import org.apache.spark.sql.{CarbonEnv, DataFrame, Row, SQLContext}
+import org.apache.spark.sql.execution.command.{AlterTableModel, CompactionCallableModel, CompactionModel}
+import org.apache.spark.sql.hive.DistributionUtil
+import org.apache.spark.util.SparkUtil
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.carbon.{CarbonDataLoadSchema, CarbonTableIdentifier}
+import org.apache.carbondata.core.carbon.datastore.block.{Distributable, TableBlockInfo}
+import org.apache.carbondata.core.carbon.metadata.CarbonMetadata
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.load.{BlockDetails, LoadMetadataDetails}
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.lcm.locks.{CarbonLockFactory, CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.lcm.status.SegmentStatusManager
+import org.apache.carbondata.processing.etl.DataLoadingException
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import org.apache.carbondata.spark._
+import org.apache.carbondata.spark.load._
+import org.apache.carbondata.spark.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionCallable, CompactionType}
+import org.apache.carbondata.spark.splits.TableSplit
+import org.apache.carbondata.spark.util.{CarbonQueryUtil, LoadMetadataUtil}
+
+
+/**
+ * This is the factory class which can create different RDD depends on user needs.
+ *
+ */
+object CarbonDataRDDFactory {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def mergeCarbonData(
+      sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      storeLocation: String,
+      storePath: String) {
+    val table = CarbonMetadata.getInstance()
+      .getCarbonTable(carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName)
+    val metaDataPath: String = table.getMetaDataFilepath
+  }
+
+  def deleteLoadByDate(
+      sqlContext: SQLContext,
+      schema: CarbonDataLoadSchema,
+      databaseName: String,
+      tableName: String,
+      storePath: String,
+      dateField: String,
+      dateFieldActualName: String,
+      dateValue: String) {
+    val sc = sqlContext
+    // Delete the records based on data
+    val table = org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
+      .getCarbonTable(databaseName + "_" + tableName)
+    val loadMetadataDetailsArray =
+      SegmentStatusManager.readLoadMetadata(table.getMetaDataFilepath).toList
+
+    val resultMap = new CarbonDeleteLoadByDateRDD(
+      sc.sparkContext,
+      new DeletedLoadResultImpl(),
+      databaseName,
+      table.getDatabaseName,
+      dateField,
+      dateFieldActualName,
+      dateValue,
+      table.getFactTableName,
+      tableName,
+      storePath,
+      loadMetadataDetailsArray).collect.groupBy(_._1)
+
+    var updatedLoadMetadataDetailsList = new ListBuffer[LoadMetadataDetails]()
+    if (resultMap.nonEmpty) {
+      if (resultMap.size == 1) {
+        if (resultMap.contains("")) {
+          LOGGER.error("Delete by Date request is failed")
+          sys.error("Delete by Date request is failed, potential causes " +
+                    "Empty store or Invalid column type, For more details please refer logs.")
+        }
+      }
+      val updatedloadMetadataDetails = loadMetadataDetailsArray.map { elem => {
+        var statusList = resultMap.get(elem.getLoadName)
+        // check for the merged load folder.
+        if (statusList.isEmpty && null != elem.getMergedLoadName) {
+          statusList = resultMap.get(elem.getMergedLoadName)
+        }
+
+        if (statusList.isDefined) {
+          elem.setModificationOrdeletionTimesStamp(CarbonLoaderUtil.readCurrentTime())
+          // if atleast on CarbonCommonConstants.MARKED_FOR_UPDATE status exist,
+          // use MARKED_FOR_UPDATE
+          if (statusList.get
+            .forall(status => status._2 == CarbonCommonConstants.MARKED_FOR_DELETE)) {
+            elem.setLoadStatus(CarbonCommonConstants.MARKED_FOR_DELETE)
+          } else {
+            elem.setLoadStatus(CarbonCommonConstants.MARKED_FOR_UPDATE)
+            updatedLoadMetadataDetailsList += elem
+          }
+          elem
+        } else {
+          elem
+        }
+      }
+
+      }
+
+      // Save the load metadata
+      val carbonLock = CarbonLockFactory
+        .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+          LockUsage.METADATA_LOCK
+        )
+      try {
+        if (carbonLock.lockWithRetries()) {
+          LOGGER.info("Successfully got the table metadata file lock")
+          if (updatedLoadMetadataDetailsList.nonEmpty) {
+            // TODO: Load Aggregate tables after retention.
+          }
+
+          // write
+          CarbonLoaderUtil.writeLoadMetadata(
+            schema,
+            databaseName,
+            table.getDatabaseName,
+            updatedloadMetadataDetails.asJava
+          )
+        }
+      } finally {
+        if (carbonLock.unlock()) {
+          LOGGER.info("unlock the table metadata file successfully")
+        } else {
+          LOGGER.error("Unable to unlock the metadata lock")
+        }
+      }
+    } else {
+      LOGGER.error("Delete by Date request is failed")
+      LOGGER.audit(s"The delete load by date is failed for $databaseName.$tableName")
+      sys.error("Delete by Date request is failed, potential causes " +
+                "Empty store or Invalid column type, For more details please refer logs.")
+    }
+  }
+
+  def alterTableForCompaction(sqlContext: SQLContext,
+      alterTableModel: AlterTableModel,
+      carbonLoadModel: CarbonLoadModel, storePath: String,
+      kettleHomePath: String, storeLocation: String): Unit = {
+    var compactionSize: Long = 0
+    var compactionType: CompactionType = CompactionType.MINOR_COMPACTION
+    if (alterTableModel.compactionType.equalsIgnoreCase("major")) {
+      compactionSize = CarbonDataMergerUtil.getCompactionSize(CompactionType.MAJOR_COMPACTION)
+      compactionType = CompactionType.MAJOR_COMPACTION
+    } else {
+      compactionType = CompactionType.MINOR_COMPACTION
+    }
+
+    LOGGER.audit(s"Compaction request received for table " +
+                 s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    val tableCreationTime = CarbonEnv.get.carbonMetastore
+      .getTableCreationTime(carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
+
+    if (null == carbonLoadModel.getLoadMetadataDetails) {
+      readLoadMetadataDetails(carbonLoadModel, storePath)
+    }
+    // reading the start time of data load.
+    val loadStartTime = CarbonLoaderUtil.readCurrentTime()
+    carbonLoadModel.setFactTimeStamp(loadStartTime)
+
+    val isCompactionTriggerByDDl = true
+    val compactionModel = CompactionModel(compactionSize,
+      compactionType,
+      carbonTable,
+      tableCreationTime,
+      isCompactionTriggerByDDl
+    )
+
+    val isConcurrentCompactionAllowed = CarbonProperties.getInstance()
+      .getProperty(CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
+        CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
+      )
+      .equalsIgnoreCase("true")
+
+    // if system level compaction is enabled then only one compaction can run in the system
+    // if any other request comes at this time then it will create a compaction request file.
+    // so that this will be taken up by the compaction process which is executing.
+    if (!isConcurrentCompactionAllowed) {
+      LOGGER.info("System level compaction lock is enabled.")
+      handleCompactionForSystemLocking(sqlContext,
+        carbonLoadModel,
+        storePath,
+        kettleHomePath,
+        storeLocation,
+        compactionType,
+        carbonTable,
+        compactionModel
+      )
+    } else {
+      // normal flow of compaction
+      val lock = CarbonLockFactory
+        .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+          LockUsage.COMPACTION_LOCK
+        )
+
+      if (lock.lockWithRetries()) {
+        LOGGER.info("Acquired the compaction lock for table" +
+                    s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        try {
+          startCompactionThreads(sqlContext,
+            carbonLoadModel,
+            storePath,
+            kettleHomePath,
+            storeLocation,
+            compactionModel,
+            lock
+          )
+        } catch {
+          case e: Exception =>
+            LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
+            lock.unlock()
+        }
+      } else {
+        LOGGER.audit("Not able to acquire the compaction lock for table " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        LOGGER.error(s"Not able to acquire the compaction lock for table" +
+                     s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        sys.error("Table is already locked for compaction. Please try after some time.")
+      }
+    }
+  }
+
+  def handleCompactionForSystemLocking(sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      storePath: String,
+      kettleHomePath: String,
+      storeLocation: String,
+      compactionType: CompactionType,
+      carbonTable: CarbonTable,
+      compactionModel: CompactionModel): Unit = {
+    val lock = CarbonLockFactory
+      .getCarbonLockObj(CarbonCommonConstants.SYSTEM_LEVEL_COMPACTION_LOCK_FOLDER,
+        LockUsage.SYSTEMLEVEL_COMPACTION_LOCK
+      )
+    if (lock.lockWithRetries()) {
+      LOGGER.info(s"Acquired the compaction lock for table ${ carbonLoadModel.getDatabaseName }" +
+                  s".${ carbonLoadModel.getTableName }")
+      try {
+        startCompactionThreads(sqlContext,
+          carbonLoadModel,
+          storePath,
+          kettleHomePath,
+          storeLocation,
+          compactionModel,
+          lock
+        )
+      } catch {
+        case e: Exception =>
+          LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
+          lock.unlock()
+          // if the compaction is a blocking call then only need to throw the exception.
+          if (compactionModel.isDDLTrigger) {
+            throw e
+          }
+      }
+    } else {
+      LOGGER.audit("Not able to acquire the system level compaction lock for table " +
+                   s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      LOGGER.error("Not able to acquire the compaction lock for table " +
+                   s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      CarbonCompactionUtil
+        .createCompactionRequiredFile(carbonTable.getMetaDataFilepath, compactionType)
+      // do sys error only in case of DDL trigger.
+      if (compactionModel.isDDLTrigger) {
+        sys.error("Compaction is in progress, compaction request for table " +
+                  s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }" +
+                  " is in queue.")
+      } else {
+        LOGGER.error("Compaction is in progress, compaction request for table " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }" +
+                     " is in queue.")
+      }
+    }
+  }
+
+  def executeCompaction(carbonLoadModel: CarbonLoadModel,
+      storePath: String,
+      compactionModel: CompactionModel,
+      executor: ExecutorService,
+      sqlContext: SQLContext,
+      kettleHomePath: String,
+      storeLocation: String): Unit = {
+    val sortedSegments: util.List[LoadMetadataDetails] = new util.ArrayList[LoadMetadataDetails](
+      carbonLoadModel.getLoadMetadataDetails
+    )
+    CarbonDataMergerUtil.sortSegments(sortedSegments)
+
+    var segList = carbonLoadModel.getLoadMetadataDetails
+    var loadsToMerge = CarbonDataMergerUtil.identifySegmentsToBeMerged(
+      storePath,
+      carbonLoadModel,
+      compactionModel.compactionSize,
+      segList,
+      compactionModel.compactionType
+    )
+    while (loadsToMerge.size() > 1) {
+      val lastSegment = sortedSegments.get(sortedSegments.size() - 1)
+      deletePartialLoadsInCompaction(carbonLoadModel)
+      val futureList: util.List[Future[Void]] = new util.ArrayList[Future[Void]](
+        CarbonCommonConstants
+          .DEFAULT_COLLECTION_SIZE
+      )
+
+      scanSegmentsAndSubmitJob(futureList,
+        loadsToMerge,
+        executor,
+        storePath,
+        sqlContext,
+        compactionModel,
+        kettleHomePath,
+        carbonLoadModel,
+        storeLocation
+      )
+
+      try {
+
+        futureList.asScala.foreach(future => {
+          future.get
+        }
+        )
+      } catch {
+        case e: Exception =>
+          LOGGER.error(s"Exception in compaction thread ${ e.getMessage }")
+          throw e
+      }
+
+
+      // scan again and determine if anything is there to merge again.
+      readLoadMetadataDetails(carbonLoadModel, storePath)
+      segList = carbonLoadModel.getLoadMetadataDetails
+      // in case of major compaction we will scan only once and come out as it will keep
+      // on doing major for the new loads also.
+      // excluding the newly added segments.
+      if (compactionModel.compactionType == CompactionType.MAJOR_COMPACTION) {
+
+        segList = CarbonDataMergerUtil
+          .filterOutNewlyAddedSegments(carbonLoadModel.getLoadMetadataDetails, lastSegment)
+      }
+      loadsToMerge = CarbonDataMergerUtil.identifySegmentsToBeMerged(
+        storePath,
+        carbonLoadModel,
+        compactionModel.compactionSize,
+        segList,
+        compactionModel.compactionType
+      )
+    }
+  }
+
+  /**
+   * This will submit the loads to be merged into the executor.
+   *
+   * @param futureList
+   */
+  def scanSegmentsAndSubmitJob(futureList: util.List[Future[Void]],
+      loadsToMerge: util
+      .List[LoadMetadataDetails],
+      executor: ExecutorService,
+      storePath: String,
+      sqlContext: SQLContext,
+      compactionModel: CompactionModel,
+      kettleHomePath: String,
+      carbonLoadModel: CarbonLoadModel,
+      storeLocation: String): Unit = {
+
+    loadsToMerge.asScala.foreach(seg => {
+      LOGGER.info("loads identified for merge is " + seg.getLoadName)
+    }
+    )
+
+    val compactionCallableModel = CompactionCallableModel(storePath,
+      carbonLoadModel,
+      storeLocation,
+      compactionModel.carbonTable,
+      kettleHomePath,
+      compactionModel.tableCreationTime,
+      loadsToMerge,
+      sqlContext,
+      compactionModel.compactionType
+    )
+
+    val future: Future[Void] = executor
+      .submit(new CompactionCallable(compactionCallableModel
+      )
+      )
+    futureList.add(future)
+  }
+
+  def startCompactionThreads(sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      storePath: String,
+      kettleHomePath: String,
+      storeLocation: String,
+      compactionModel: CompactionModel,
+      compactionLock: ICarbonLock): Unit = {
+    val executor: ExecutorService = Executors.newFixedThreadPool(1)
+    // update the updated table status.
+    readLoadMetadataDetails(carbonLoadModel, storePath)
+    var segList: util.List[LoadMetadataDetails] = carbonLoadModel.getLoadMetadataDetails
+
+    // clean up of the stale segments.
+    try {
+      CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, true)
+    } catch {
+      case e: Exception =>
+        LOGGER.error(s"Exception in compaction thread while clean up of stale segments" +
+                     s" ${ e.getMessage }")
+    }
+
+    val compactionThread = new Thread {
+      override def run(): Unit = {
+
+        try {
+          // compaction status of the table which is triggered by the user.
+          var triggeredCompactionStatus = false
+          var exception: Exception = null
+          try {
+            executeCompaction(carbonLoadModel: CarbonLoadModel,
+              storePath: String,
+              compactionModel: CompactionModel,
+              executor, sqlContext, kettleHomePath, storeLocation
+            )
+            triggeredCompactionStatus = true
+          } catch {
+            case e: Exception =>
+              LOGGER.error(s"Exception in compaction thread ${ e.getMessage }")
+              exception = e
+          }
+          // continue in case of exception also, check for all the tables.
+          val isConcurrentCompactionAllowed = CarbonProperties.getInstance()
+            .getProperty(CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
+              CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
+            ).equalsIgnoreCase("true")
+
+          if (!isConcurrentCompactionAllowed) {
+            LOGGER.info("System level compaction lock is enabled.")
+            val skipCompactionTables = ListBuffer[CarbonTableIdentifier]()
+            var tableForCompaction = CarbonCompactionUtil.getNextTableToCompact(
+              CarbonEnv.get.carbonMetastore.metadata.tablesMeta.toArray,
+              skipCompactionTables.toList.asJava)
+            while (null != tableForCompaction) {
+              LOGGER.info("Compaction request has been identified for table " +
+                          s"${ tableForCompaction.carbonTable.getDatabaseName }." +
+                          s"${ tableForCompaction.carbonTableIdentifier.getTableName }")
+              val table: CarbonTable = tableForCompaction.carbonTable
+              val metadataPath = table.getMetaDataFilepath
+              val compactionType = CarbonCompactionUtil.determineCompactionType(metadataPath)
+
+              val newCarbonLoadModel = new CarbonLoadModel()
+              prepareCarbonLoadModel(storePath, table, newCarbonLoadModel)
+              val tableCreationTime = CarbonEnv.get.carbonMetastore
+                .getTableCreationTime(newCarbonLoadModel.getDatabaseName,
+                  newCarbonLoadModel.getTableName
+                )
+
+              val compactionSize = CarbonDataMergerUtil
+                .getCompactionSize(CompactionType.MAJOR_COMPACTION)
+
+              val newcompactionModel = CompactionModel(compactionSize,
+                compactionType,
+                table,
+                tableCreationTime,
+                compactionModel.isDDLTrigger
+              )
+              // proceed for compaction
+              try {
+                executeCompaction(newCarbonLoadModel,
+                  newCarbonLoadModel.getStorePath,
+                  newcompactionModel,
+                  executor, sqlContext, kettleHomePath, storeLocation
+                )
+              } catch {
+                case e: Exception =>
+                  LOGGER.error("Exception in compaction thread for table " +
+                               s"${ tableForCompaction.carbonTable.getDatabaseName }." +
+                               s"${ tableForCompaction.carbonTableIdentifier.getTableName }")
+                // not handling the exception. only logging as this is not the table triggered
+                // by user.
+              } finally {
+                // delete the compaction required file in case of failure or success also.
+                if (!CarbonCompactionUtil
+                  .deleteCompactionRequiredFile(metadataPath, compactionType)) {
+                  // if the compaction request file is not been able to delete then
+                  // add those tables details to the skip list so that it wont be considered next.
+                  skipCompactionTables.+=:(tableForCompaction.carbonTableIdentifier)
+                  LOGGER.error("Compaction request file can not be deleted for table " +
+                               s"${ tableForCompaction.carbonTable.getDatabaseName }." +
+                               s"${ tableForCompaction.carbonTableIdentifier.getTableName }")
+                }
+              }
+              // ********* check again for all the tables.
+              tableForCompaction = CarbonCompactionUtil
+                .getNextTableToCompact(CarbonEnv.get.carbonMetastore.metadata
+                  .tablesMeta.toArray, skipCompactionTables.asJava
+                )
+            }
+            // giving the user his error for telling in the beeline if his triggered table
+            // compaction is failed.
+            if (!triggeredCompactionStatus) {
+              throw new Exception("Exception in compaction " + exception.getMessage)
+            }
+          }
+        } finally {
+          executor.shutdownNow()
+          deletePartialLoadsInCompaction(carbonLoadModel)
+          compactionLock.unlock()
+        }
+      }
+    }
+    // calling the run method of a thread to make the call as blocking call.
+    // in the future we may make this as concurrent.
+    compactionThread.run()
+  }
+
+  def prepareCarbonLoadModel(storePath: String,
+      table: CarbonTable,
+      newCarbonLoadModel: CarbonLoadModel): Unit = {
+    newCarbonLoadModel.setAggTables(table.getAggregateTablesName.asScala.toArray)
+    newCarbonLoadModel.setTableName(table.getFactTableName)
+    val dataLoadSchema = new CarbonDataLoadSchema(table)
+    // Need to fill dimension relation
+    newCarbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+    newCarbonLoadModel.setTableName(table.getCarbonTableIdentifier.getTableName)
+    newCarbonLoadModel.setDatabaseName(table.getCarbonTableIdentifier.getDatabaseName)
+    newCarbonLoadModel.setStorePath(table.getStorePath)
+    readLoadMetadataDetails(newCarbonLoadModel, storePath)
+    val loadStartTime = CarbonLoaderUtil.readCurrentTime()
+    newCarbonLoadModel.setFactTimeStamp(loadStartTime)
+  }
+
+  def deletePartialLoadsInCompaction(carbonLoadModel: CarbonLoadModel): Unit = {
+    // Deleting the any partially loaded data if present.
+    // in some case the segment folder which is present in store will not have entry in
+    // status.
+    // so deleting those folders.
+    try {
+      CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, true)
+    } catch {
+      case e: Exception =>
+        LOGGER.error(s"Exception in compaction thread while clean up of stale segments" +
+                     s" ${ e.getMessage }")
+    }
+  }
+
+  def loadCarbonData(sqlContext: SQLContext,
+      carbonLoadModel: CarbonLoadModel,
+      storePath: String,
+      kettleHomePath: String,
+      columinar: Boolean,
+      partitionStatus: String = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS,
+      useKettle: Boolean,
+      dataFrame: Option[DataFrame] = None): Unit = {
+    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+    val isAgg = false
+    // for handling of the segment Merging.
+    def handleSegmentMerging(tableCreationTime: Long): Unit = {
+      LOGGER.info(s"compaction need status is" +
+                  s" ${ CarbonDataMergerUtil.checkIfAutoLoadMergingRequired() }")
+      if (CarbonDataMergerUtil.checkIfAutoLoadMergingRequired()) {
+        LOGGER.audit(s"Compaction request received for table " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        val compactionSize = 0
+        val isCompactionTriggerByDDl = false
+        val compactionModel = CompactionModel(compactionSize,
+          CompactionType.MINOR_COMPACTION,
+          carbonTable,
+          tableCreationTime,
+          isCompactionTriggerByDDl
+        )
+        var storeLocation = ""
+        val configuredStore = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
+        if (null != configuredStore && configuredStore.nonEmpty) {
+          storeLocation = configuredStore(Random.nextInt(configuredStore.length))
+        }
+        if (storeLocation == null) {
+          storeLocation = System.getProperty("java.io.tmpdir")
+        }
+        storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
+
+        val isConcurrentCompactionAllowed = CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
+            CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
+          )
+          .equalsIgnoreCase("true")
+
+        if (!isConcurrentCompactionAllowed) {
+
+          handleCompactionForSystemLocking(sqlContext,
+            carbonLoadModel,
+            storePath,
+            kettleHomePath,
+            storeLocation,
+            CompactionType.MINOR_COMPACTION,
+            carbonTable,
+            compactionModel
+          )
+        } else {
+          val lock = CarbonLockFactory
+            .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+              LockUsage.COMPACTION_LOCK
+            )
+
+          if (lock.lockWithRetries()) {
+            LOGGER.info("Acquired the compaction lock.")
+            try {
+              startCompactionThreads(sqlContext,
+                carbonLoadModel,
+                storePath,
+                kettleHomePath,
+                storeLocation,
+                compactionModel,
+                lock
+              )
+            } catch {
+              case e: Exception =>
+                LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
+                lock.unlock()
+                throw e
+            }
+          } else {
+            LOGGER.audit("Not able to acquire the compaction lock for table " +
+                         s"${ carbonLoadModel.getDatabaseName }.${
+                           carbonLoadModel
+                             .getTableName
+                         }")
+            LOGGER.error("Not able to acquire the compaction lock for table " +
+                         s"${ carbonLoadModel.getDatabaseName }.${
+                           carbonLoadModel
+                             .getTableName
+                         }")
+          }
+        }
+      }
+    }
+
+    try {
+      LOGGER.audit(s"Data load request has been received for table" +
+                   s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      if (!useKettle) {
+        LOGGER.audit("Data is loading with New Data Flow for table " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+      }
+      // Check if any load need to be deleted before loading new data
+      deleteLoadsAndUpdateMetadata(carbonLoadModel, carbonTable, storePath,
+        isForceDeletion = false)
+      if (null == carbonLoadModel.getLoadMetadataDetails) {
+        readLoadMetadataDetails(carbonLoadModel, storePath)
+      }
+
+      var currentLoadCount = -1
+      val convLoadDetails = carbonLoadModel.getLoadMetadataDetails.asScala
+      // taking the latest segment ID present.
+      // so that any other segments above this will be deleted.
+      if (convLoadDetails.nonEmpty) {
+        convLoadDetails.foreach { l =>
+          var loadCount = 0
+          breakable {
+            try {
+              loadCount = Integer.parseInt(l.getLoadName)
+            } catch {
+              case e: NumberFormatException => // case of merge folder. ignore it.
+                break
+            }
+            if (currentLoadCount < loadCount) {
+              currentLoadCount = loadCount
+            }
+          }
+        }
+      }
+      currentLoadCount += 1
+      // Deleting the any partially loaded data if present.
+      // in some case the segment folder which is present in store will not have entry in status.
+      // so deleting those folders.
+      try {
+        CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, false)
+      } catch {
+        case e: Exception =>
+          LOGGER
+            .error(s"Exception in data load while clean up of stale segments ${ e.getMessage }")
+      }
+
+      // reading the start time of data load.
+      val loadStartTime = CarbonLoaderUtil.readCurrentTime()
+      carbonLoadModel.setFactTimeStamp(loadStartTime)
+      val tableCreationTime = CarbonEnv.get.carbonMetastore
+        .getTableCreationTime(carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
+      val schemaLastUpdatedTime = CarbonEnv.get.carbonMetastore
+        .getSchemaLastUpdatedTime(carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
+
+      // get partition way from configuration
+      // val isTableSplitPartition = CarbonProperties.getInstance().getProperty(
+      // CarbonCommonConstants.TABLE_SPLIT_PARTITION,
+      // CarbonCommonConstants.TABLE_SPLIT_PARTITION_DEFAULT_VALUE).toBoolean
+      val isTableSplitPartition = false
+      var blocksGroupBy: Array[(String, Array[BlockDetails])] = null
+      var status: Array[(String, LoadMetadataDetails)] = null
+
+      def loadDataFile(): Unit = {
+        if (isTableSplitPartition) {
+          /*
+         * when data handle by table split partition
+         * 1) get partition files, direct load or not will get the different files path
+         * 2) get files blocks by using SplitUtils
+         * 3) output Array[(partitionID,Array[BlockDetails])] to blocksGroupBy
+         */
+          var splits = Array[TableSplit]()
+          if (carbonLoadModel.isDirectLoad) {
+            // get all table Splits, this part means files were divide to different partitions
+            splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
+            // get all partition blocks from file list
+            blocksGroupBy = splits.map {
+              split =>
+                val pathBuilder = new StringBuilder()
+                for (path <- split.getPartition.getFilesPath.asScala) {
+                  pathBuilder.append(path).append(",")
+                }
+                if (pathBuilder.nonEmpty) {
+                  pathBuilder.substring(0, pathBuilder.size - 1)
+                }
+                (split.getPartition.getUniqueID, SparkUtil.getSplits(pathBuilder.toString(),
+                  sqlContext.sparkContext
+                ))
+            }
+          } else {
+            // get all table Splits,when come to this, means data have been partition
+            splits = CarbonQueryUtil.getTableSplits(carbonLoadModel.getDatabaseName,
+              carbonLoadModel.getTableName, null
+            )
+            // get all partition blocks from factFilePath/uniqueID/
+            blocksGroupBy = splits.map {
+              split =>
+                val pathBuilder = new StringBuilder()
+                pathBuilder.append(carbonLoadModel.getFactFilePath)
+                if (!carbonLoadModel.getFactFilePath.endsWith("/")
+                    && !carbonLoadModel.getFactFilePath.endsWith("\\")) {
+                  pathBuilder.append("/")
+                }
+                pathBuilder.append(split.getPartition.getUniqueID).append("/")
+                (split.getPartition.getUniqueID,
+                    SparkUtil.getSplits(pathBuilder.toString, sqlContext.sparkContext))
+            }
+          }
+        } else {
+          /*
+         * when data load handle by node partition
+         * 1)clone the hadoop configuration,and set the file path to the configuration
+         * 2)use NewHadoopRDD to get split,size:Math.max(minSize, Math.min(maxSize, blockSize))
+         * 3)use DummyLoadRDD to group blocks by host,and let spark balance the block location
+         * 4)DummyLoadRDD output (host,Array[BlockDetails])as the parameter to CarbonDataLoadRDD
+         *   which parititon by host
+         */
+          val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
+          // FileUtils will skip file which is no csv, and return all file path which split by ','
+          val filePaths = carbonLoadModel.getFactFilePath
+          hadoopConfiguration.set(FileInputFormat.INPUT_DIR, filePaths)
+          hadoopConfiguration.set(FileInputFormat.INPUT_DIR_RECURSIVE, "true")
+          hadoopConfiguration.set("io.compression.codecs",
+            """org.apache.hadoop.io.compress.GzipCodec,
+               org.apache.hadoop.io.compress.DefaultCodec,
+               org.apache.hadoop.io.compress.BZip2Codec""".stripMargin)
+
+          CarbonTextFile.configSplitMaxSize(sqlContext.sparkContext, filePaths, hadoopConfiguration)
+
+          val inputFormat = new org.apache.hadoop.mapreduce.lib.input.TextInputFormat
+          inputFormat match {
+            case configurable: Configurable =>
+              configurable.setConf(hadoopConfiguration)
+            case _ =>
+          }
+          val jobContext = new Job(hadoopConfiguration)
+          val rawSplits = inputFormat.getSplits(jobContext).toArray
+          val blockList = rawSplits.map(inputSplit => {
+            val fileSplit = inputSplit.asInstanceOf[FileSplit]
+            new TableBlockInfo(fileSplit.getPath.toString,
+              fileSplit.getStart, "1",
+              fileSplit.getLocations, fileSplit.getLength
+            ).asInstanceOf[Distributable]
+          }
+          )
+          // group blocks to nodes, tasks
+          val startTime = System.currentTimeMillis
+          val activeNodes = DistributionUtil
+            .ensureExecutorsAndGetNodeList(blockList, sqlContext.sparkContext)
+          val nodeBlockMapping =
+            CarbonLoaderUtil
+              .nodeBlockMapping(blockList.toSeq.asJava, -1, activeNodes.toList.asJava).asScala
+              .toSeq
+          val timeElapsed: Long = System.currentTimeMillis - startTime
+          LOGGER.info("Total Time taken in block allocation: " + timeElapsed)
+          LOGGER.info(s"Total no of blocks: ${ blockList.length }, " +
+                      s"No.of Nodes: ${nodeBlockMapping.size}")
+          var str = ""
+          nodeBlockMapping.foreach(entry => {
+            val tableBlock = entry._2
+            str = str + "#Node: " + entry._1 + " no.of.blocks: " + tableBlock.size()
+            tableBlock.asScala.foreach(tableBlockInfo =>
+              if (!tableBlockInfo.getLocations.exists(hostentry =>
+                hostentry.equalsIgnoreCase(entry._1)
+              )) {
+                str = str + " , mismatch locations: " + tableBlockInfo.getLocations
+                  .foldLeft("")((a, b) => a + "," + b)
+              }
+            )
+            str = str + "\n"
+          }
+          )
+          LOGGER.info(str)
+          blocksGroupBy = nodeBlockMapping.map(entry => {
+            val blockDetailsList =
+              entry._2.asScala.map(distributable => {
+                val tableBlock = distributable.asInstanceOf[TableBlockInfo]
+                new BlockDetails(new Path(tableBlock.getFilePath),
+                  tableBlock.getBlockOffset, tableBlock.getBlockLength, tableBlock.getLocations
+                )
+              }).toArray
+            (entry._1, blockDetailsList)
+          }
+          ).toArray
+        }
+
+        if (useKettle) {
+          status = new DataFileLoaderRDD(sqlContext.sparkContext,
+            new DataLoadResultImpl(),
+            carbonLoadModel,
+            storePath,
+            kettleHomePath,
+            columinar,
+            currentLoadCount,
+            tableCreationTime,
+            schemaLastUpdatedTime,
+            blocksGroupBy,
+            isTableSplitPartition
+          ).collect()
+        } else {
+          status = new NewCarbonDataLoadRDD(sqlContext.sparkContext,
+            new DataLoadResultImpl(),
+            carbonLoadModel,
+            currentLoadCount,
+            blocksGroupBy,
+            isTableSplitPartition).collect()
+        }
+      }
+
+      def loadDataFrame(): Unit = {
+        try {
+          val rdd = dataFrame.get.rdd
+          val nodeNumOfData = rdd.partitions.flatMap[String, Array[String]]{ p =>
+            DataLoadPartitionCoalescer.getPreferredLocs(rdd, p).map(_.host)
+          }.distinct.size
+          val nodes = DistributionUtil.ensureExecutorsByNumberAndGetNodeList(nodeNumOfData,
+            sqlContext.sparkContext)
+          val newRdd = new DataLoadCoalescedRDD[Row](rdd, nodes.toArray.distinct)
+
+          status = new DataFrameLoaderRDD(sqlContext.sparkContext,
+            new DataLoadResultImpl(),
+            carbonLoadModel,
+            storePath,
+            kettleHomePath,
+            columinar,
+            currentLoadCount,
+            tableCreationTime,
+            schemaLastUpdatedTime,
+            newRdd).collect()
+        } catch {
+          case ex: Exception =>
+            LOGGER.error(ex, "load data frame failed")
+            throw ex
+        }
+      }
+
+      CarbonLoaderUtil.checkAndCreateCarbonDataLocation(storePath,
+        carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName, currentLoadCount.toString)
+      var loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
+      var errorMessage: String = "DataLoad failure"
+      var executorMessage: String = ""
+      try {
+        if (dataFrame.isDefined) {
+          loadDataFrame()
+        } else {
+          loadDataFile()
+        }
+        val newStatusMap = scala.collection.mutable.Map.empty[String, String]
+        if (status.nonEmpty) {
+          status.foreach { eachLoadStatus =>
+            val state = newStatusMap.get(eachLoadStatus._1)
+            state match {
+              case Some(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) =>
+                newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2.getLoadStatus)
+              case Some(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)
+                if eachLoadStatus._2.getLoadStatus ==
+                    CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS =>
+                newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2.getLoadStatus)
+              case _ =>
+                newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2.getLoadStatus)
+            }
+          }
+
+          newStatusMap.foreach {
+            case (key, value) =>
+              if (value == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
+                loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
+              } else if (value == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
+                  !loadStatus.equals(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
+                loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
+              }
+          }
+        } else {
+          loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
+        }
+
+        if (loadStatus != CarbonCommonConstants.STORE_LOADSTATUS_FAILURE &&
+            partitionStatus == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS) {
+          loadStatus = partitionStatus
+        }
+      } catch {
+        case ex: Throwable =>
+          loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
+          ex match {
+            case sparkException: SparkException =>
+              if (sparkException.getCause.isInstanceOf[DataLoadingException] ||
+                  sparkException.getCause.isInstanceOf[CarbonDataLoadingException]) {
+                executorMessage = sparkException.getCause.getMessage
+                errorMessage = errorMessage + ": " + executorMessage
+              }
+            case _ =>
+              executorMessage = ex.getCause.getMessage
+              errorMessage = errorMessage + ": " + executorMessage
+          }
+          LOGGER.info(errorMessage)
+          LOGGER.error(ex)
+      }
+
+      if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
+        LOGGER.info("********starting clean up**********")
+        CarbonLoaderUtil.deleteSegment(carbonLoadModel, currentLoadCount)
+        LOGGER.info("********clean up done**********")
+        LOGGER.audit(s"Data load is failed for " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        LOGGER.warn("Cannot write load metadata file as data load failed")
+        throw new Exception(errorMessage)
+      } else {
+        val metadataDetails = status(0)._2
+        if (!isAgg) {
+          val status = CarbonLoaderUtil.recordLoadMetadata(currentLoadCount, metadataDetails,
+            carbonLoadModel, loadStatus, loadStartTime)
+          if (!status) {
+            val errorMessage = "Dataload failed due to failure in table status updation."
+            LOGGER.audit("Data load is failed for " +
+                         s"${ carbonLoadModel.getDatabaseName }.${
+                           carbonLoadModel
+                             .getTableName
+                         }")
+            LOGGER.error("Dataload failed due to failure in table status updation.")
+            throw new Exception(errorMessage)
+          }
+        } else if (!carbonLoadModel.isRetentionRequest) {
+          // TODO : Handle it
+          LOGGER.info("********Database updated**********")
+        }
+        LOGGER.audit("Data load is successful for " +
+                     s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
+        try {
+          // compaction handling
+          handleSegmentMerging(tableCreationTime)
+        } catch {
+          case e: Exception =>
+            throw new Exception(
+              "Dataload is success. Auto-Compaction has failed. Please check logs.")
+        }
+      }
+    }
+
+  }
+
+  def readLoadMetadataDetails(model: CarbonLoadModel, storePath: String): Unit = {
+    val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetaDataFilepath
+    val details = SegmentStatusManager.readLoadMetadata(metadataPath)
+    model.setLoadMetadataDetails(details.toList.asJava)
+  }
+
+  def deleteLoadsAndUpdateMetadata(
+      carbonLoadModel: CarbonLoadModel,
+      table: CarbonTable,
+      storePath: String,
+      isForceDeletion: Boolean): Unit = {
+    if (LoadMetadataUtil.isLoadDeletionRequired(carbonLoadModel)) {
+      val loadMetadataFilePath = CarbonLoaderUtil
+        .extractLoadMetadataFileLocation(carbonLoadModel)
+      val details = SegmentStatusManager.readLoadMetadata(loadMetadataFilePath)
+      val carbonTableStatusLock = CarbonLockFactory
+        .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+          LockUsage.TABLE_STATUS_LOCK)
+
+      // Delete marked loads
+      val isUpdationRequired = DeleteLoadFolders
+        .deleteLoadFoldersFromFileSystem(carbonLoadModel, storePath, isForceDeletion, details)
+
+      if (isUpdationRequired) {
+        try {
+          // Update load metadate file after cleaning deleted nodes
+          if (carbonTableStatusLock.lockWithRetries()) {
+            LOGGER.info("Table status lock has been successfully acquired.")
+
+            // read latest table status again.
+            val latestMetadata = SegmentStatusManager.readLoadMetadata(loadMetadataFilePath)
+
+            // update the metadata details from old to new status.
+            val latestStatus = CarbonLoaderUtil
+              .updateLoadMetadataFromOldToNew(details, latestMetadata)
+
+            CarbonLoaderUtil.writeLoadMetadata(
+              carbonLoadModel.getCarbonDataLoadSchema,
+              carbonLoadModel.getDatabaseName,
+              carbonLoadModel.getTableName, latestStatus)
+          } else {
+            val errorMsg = "Clean files request is failed for " +
+              s"${ carbonLoadModel.getDatabaseName }." +
+              s"${ carbonLoadModel.getTableName }" +
+              ". Not able to acquire the table status lock due to other operation " +
+              "running in the background."
+            LOGGER.audit(errorMsg)
+            LOGGER.error(errorMsg)
+            throw new Exception(errorMsg + " Please try after some time.")
+          }
+        } finally {
+          CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.TABLE_STATUS_LOCK)
+        }
+      }
+    }
+  }
+
+  def dropTable(
+      sc: SparkContext,
+      schema: String,
+      table: String) {
+    val v: Value[Array[Object]] = new ValueImpl()
+    new CarbonDropTableRDD(sc, v, schema, table).collect
+  }
+
+  def cleanFiles(
+      sc: SparkContext,
+      carbonLoadModel: CarbonLoadModel,
+      storePath: String) {
+    val table = org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
+      .getCarbonTable(carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName)
+    val metaDataPath: String = table.getMetaDataFilepath
+    val carbonCleanFilesLock = CarbonLockFactory
+      .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+        LockUsage.CLEAN_FILES_LOCK
+      )
+    try {
+      if (carbonCleanFilesLock.lockWithRetries()) {
+        LOGGER.info("Clean files lock has been successfully acquired.")
+        deleteLoadsAndUpdateMetadata(carbonLoadModel,
+          table,
+          storePath,
+          isForceDeletion = true)
+      } else {
+        val errorMsg = "Clean files request is failed for " +
+                       s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }" +
+                       ". Not able to acquire the clean files lock due to another clean files " +
+                       "operation is running in the background."
+        LOGGER.audit(errorMsg)
+        LOGGER.error(errorMsg)
+        throw new Exception(errorMsg + " Please try after some time.")
+
+      }
+    } finally {
+      CarbonLockUtil.fileUnlock(carbonCleanFilesLock, LockUsage.CLEAN_FILES_LOCK)
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
new file mode 100644
index 0000000..dac9098
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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 scala.collection.JavaConverters._
+import scala.reflect.ClassTag
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.{InputSplit, Job, JobID, TaskAttemptID, TaskType}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext, TaskKilledException}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.hive.DistributionUtil
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier
+import org.apache.carbondata.core.carbon.datastore.block.Distributable
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants}
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory
+import org.apache.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit, CarbonMultiBlockSplit, CarbonProjection}
+import org.apache.carbondata.hadoop.readsupport.impl.RawDataReadSupport
+import org.apache.carbondata.scan.expression.Expression
+import org.apache.carbondata.spark.load.CarbonLoaderUtil
+import org.apache.carbondata.spark.readsupport.SparkRowReadSupportImpl
+
+class CarbonSparkPartition(
+    val rddId: Int,
+    val idx: Int,
+    @transient val multiBlockSplit: CarbonMultiBlockSplit)
+  extends Partition {
+
+  val split = new SerializableWritable[CarbonMultiBlockSplit](multiBlockSplit)
+
+  override val index: Int = idx
+
+  override def hashCode(): Int = 41 * (41 + rddId) + idx
+}
+
+/**
+ * This RDD is used to perform query on CarbonData file. Before sending tasks to scan
+ * CarbonData file, this RDD will leverage CarbonData's index information to do CarbonData file
+ * level filtering in driver side.
+ */
+class CarbonScanRDD[V: ClassTag](
+    @transient sc: SparkContext,
+    columnProjection: CarbonProjection,
+    filterExpression: Expression,
+    identifier: AbsoluteTableIdentifier,
+    @transient carbonTable: CarbonTable)
+  extends RDD[V](sc, Nil) {
+
+  private val queryId = sparkContext.getConf.get("queryId", System.nanoTime() + "")
+  private val jobTrackerId: String = {
+    val formatter = new SimpleDateFormat("yyyyMMddHHmm")
+    formatter.format(new Date())
+  }
+
+  @transient private val jobId = new JobID(jobTrackerId, id)
+  @transient val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+
+  override def getPartitions: Array[Partition] = {
+    val job = Job.getInstance(new Configuration())
+    val format = prepareInputFormatForDriver(job.getConfiguration)
+
+    // initialise query_id for job
+    job.getConfiguration.set("query.id", queryId)
+
+    // get splits
+    val splits = format.getSplits(job)
+    val result = distributeSplits(splits)
+    result
+  }
+
+  private def distributeSplits(splits: util.List[InputSplit]): Array[Partition] = {
+    // this function distributes the split based on following logic:
+    // 1. based on data locality, to make split balanced on all available nodes
+    // 2. if the number of split for one
+
+    var statistic = new QueryStatistic()
+    val statisticRecorder = CarbonTimeStatisticsFactory.createDriverRecorder()
+    val parallelism = sparkContext.defaultParallelism
+    val result = new util.ArrayList[Partition](parallelism)
+    var noOfBlocks = 0
+    var noOfNodes = 0
+    var noOfTasks = 0
+
+    if (!splits.isEmpty) {
+      // create a list of block based on split
+      val blockList = splits.asScala.map(_.asInstanceOf[Distributable])
+
+      // get the list of executors and map blocks to executors based on locality
+      val activeNodes = DistributionUtil.ensureExecutorsAndGetNodeList(blockList, sparkContext)
+
+      // divide the blocks among the tasks of the nodes as per the data locality
+      val nodeBlockMapping = CarbonLoaderUtil.nodeBlockTaskMapping(blockList.asJava, -1,
+        parallelism, activeNodes.toList.asJava)
+
+      statistic.addStatistics(QueryStatisticsConstants.BLOCK_ALLOCATION, System.currentTimeMillis)
+      statisticRecorder.recordStatisticsForDriver(statistic, queryId)
+      statistic = new QueryStatistic()
+
+      var i = 0
+      // Create Spark Partition for each task and assign blocks
+      nodeBlockMapping.asScala.foreach { case (node, blockList) =>
+        blockList.asScala.foreach { blocksPerTask =>
+          val splits = blocksPerTask.asScala.map(_.asInstanceOf[CarbonInputSplit])
+          if (blocksPerTask.size() != 0) {
+            val multiBlockSplit = new CarbonMultiBlockSplit(identifier, splits.asJava, node)
+            val partition = new CarbonSparkPartition(id, i, multiBlockSplit)
+            result.add(partition)
+            i += 1
+          }
+        }
+      }
+
+      noOfBlocks = splits.size
+      noOfNodes = nodeBlockMapping.size
+      noOfTasks = result.size()
+
+      statistic = new QueryStatistic()
+      statistic.addStatistics(QueryStatisticsConstants.BLOCK_IDENTIFICATION,
+        System.currentTimeMillis)
+      statisticRecorder.recordStatisticsForDriver(statistic, queryId)
+      statisticRecorder.logStatisticsAsTableDriver()
+    }
+    logInfo(
+      s"""
+         | Identified no.of.blocks: $noOfBlocks,
+         | no.of.tasks: $noOfTasks,
+         | no.of.nodes: $noOfNodes,
+         | parallelism: $parallelism
+       """.stripMargin)
+    result.toArray(new Array[Partition](result.size()))
+  }
+
+  override def compute(split: Partition, context: TaskContext): Iterator[V] = {
+    val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null)
+    if (null == carbonPropertiesFilePath) {
+      System.setProperty("carbon.properties.filepath",
+        System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties"
+      )
+    }
+
+    val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0)
+    val attemptContext = new TaskAttemptContextImpl(new Configuration(), attemptId)
+    val format = prepareInputFormatForExecutor(attemptContext.getConfiguration)
+    val inputSplit = split.asInstanceOf[CarbonSparkPartition].split.value
+    val reader = format.createRecordReader(inputSplit, attemptContext)
+    reader.initialize(inputSplit, attemptContext)
+
+    val queryStartTime = System.currentTimeMillis
+
+    new Iterator[V] {
+      private var havePair = false
+      private var finished = false
+      private var count = 0
+
+      context.addTaskCompletionListener { context =>
+        logStatistics(queryStartTime, count)
+        reader.close()
+      }
+
+      override def hasNext: Boolean = {
+        if (context.isInterrupted) {
+          throw new TaskKilledException
+        }
+        if (!finished && !havePair) {
+          finished = !reader.nextKeyValue
+          if (finished) {
+            reader.close()
+          }
+          havePair = !finished
+        }
+        !finished
+      }
+
+      override def next(): V = {
+        if (!hasNext) {
+          throw new java.util.NoSuchElementException("End of stream")
+        }
+        havePair = false
+        val value: V = reader.getCurrentValue
+        count += 1
+        value
+      }
+    }
+  }
+
+  private def prepareInputFormatForDriver(conf: Configuration): CarbonInputFormat[V] = {
+    CarbonInputFormat.setCarbonTable(conf, carbonTable)
+    createInputFormat(conf)
+  }
+
+  private def prepareInputFormatForExecutor(conf: Configuration): CarbonInputFormat[V] = {
+    CarbonInputFormat.setCarbonReadSupport(classOf[SparkRowReadSupportImpl], conf)
+    createInputFormat(conf)
+  }
+
+  private def createInputFormat(conf: Configuration): CarbonInputFormat[V] = {
+    val format = new CarbonInputFormat[V]
+    CarbonInputFormat.setTablePath(conf, identifier.getTablePath)
+    CarbonInputFormat.setFilterPredicates(conf, filterExpression)
+    CarbonInputFormat.setColumnProjection(conf, columnProjection)
+    format
+  }
+
+  def logStatistics(queryStartTime: Long, recordCount: Int): Unit = {
+    var queryStatistic = new QueryStatistic()
+    queryStatistic.addFixedTimeStatistic(QueryStatisticsConstants.EXECUTOR_PART,
+      System.currentTimeMillis - queryStartTime)
+    val statisticRecorder = CarbonTimeStatisticsFactory.createExecutorRecorder(queryId)
+    statisticRecorder.recordStatistics(queryStatistic)
+    // result size
+    queryStatistic = new QueryStatistic()
+    queryStatistic.addCountStatistic(QueryStatisticsConstants.RESULT_SIZE, recordCount)
+    statisticRecorder.recordStatistics(queryStatistic)
+    // print executor query statistics for each task_id
+    statisticRecorder.logStatisticsAsTableExecutor()
+  }
+
+  /**
+   * Get the preferred locations where to launch this task.
+   */
+  override def getPreferredLocations(split: Partition): Seq[String] = {
+    val theSplit = split.asInstanceOf[CarbonSparkPartition]
+    val firstOptionLocation = theSplit.split.value.getLocations.filter(_ != "localhost")
+    firstOptionLocation
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
new file mode 100644
index 0000000..914cdab
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.io.{IOException, ObjectInputStream, ObjectOutputStream}
+import java.text.SimpleDateFormat
+import java.util
+import java.util.{Date, UUID}
+
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.spark.{Partition, SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.execution.command.Partitioner
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.common.logging.impl.StandardLogService
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.load.{BlockDetails, LoadMetadataDetails}
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory
+import org.apache.carbondata.hadoop.csv.CSVInputFormat
+import org.apache.carbondata.hadoop.csv.recorditerator.RecordReaderIterator
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.newflow.DataLoadExecutor
+import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException
+import org.apache.carbondata.spark.DataLoadResult
+import org.apache.carbondata.spark.splits.TableSplit
+import org.apache.carbondata.spark.util.CarbonQueryUtil
+
+
+class SerializableConfiguration(@transient var value: Configuration) extends Serializable {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  private def writeObject(out: ObjectOutputStream): Unit =
+    try {
+      out.defaultWriteObject()
+      value.write(out)
+    } catch {
+      case e: IOException =>
+        LOGGER.error(e, "Exception encountered")
+        throw e
+      case NonFatal(e) =>
+        LOGGER.error(e, "Exception encountered")
+        throw new IOException(e)
+    }
+
+
+  private def readObject(in: ObjectInputStream): Unit =
+    try {
+      value = new Configuration(false)
+      value.readFields(in)
+    } catch {
+      case e: IOException =>
+        LOGGER.error(e, "Exception encountered")
+        throw e
+      case NonFatal(e) =>
+        LOGGER.error(e, "Exception encountered")
+        throw new IOException(e)
+    }
+}
+
+/**
+ * It loads the data to carbon using @AbstractDataLoadProcessorStep
+ */
+class NewCarbonDataLoadRDD[K, V](
+    sc: SparkContext,
+    result: DataLoadResult[K, V],
+    carbonLoadModel: CarbonLoadModel,
+    loadCount: Integer,
+    blocksGroupBy: Array[(String, Array[BlockDetails])],
+    isTableSplitPartition: Boolean)
+  extends RDD[(K, V)](sc, Nil) {
+
+  sc.setLocalProperty("spark.scheduler.pool", "DDL")
+
+  private val jobTrackerId: String = {
+    val formatter = new SimpleDateFormat("yyyyMMddHHmm")
+    formatter.format(new Date())
+  }
+
+  // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
+  private val confBroadcast =
+    sc.broadcast(new SerializableConfiguration(sc.hadoopConfiguration))
+
+  override def getPartitions: Array[Partition] = {
+    if (isTableSplitPartition) {
+      // for table split partition
+      var splits: Array[TableSplit] = null
+
+      if (carbonLoadModel.isDirectLoad) {
+        splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
+      } else {
+        splits = CarbonQueryUtil.getTableSplits(carbonLoadModel.getDatabaseName,
+          carbonLoadModel.getTableName, null)
+      }
+
+      splits.zipWithIndex.map { s =>
+        // filter the same partition unique id, because only one will match, so get 0 element
+        val blocksDetails: Array[BlockDetails] = blocksGroupBy.filter(p =>
+          p._1 == s._1.getPartition.getUniqueID)(0)._2
+        new CarbonTableSplitPartition(id, s._2, s._1, blocksDetails)
+      }
+    } else {
+      // for node partition
+      blocksGroupBy.zipWithIndex.map { b =>
+        new CarbonNodePartition(id, b._2, b._1._1, b._1._2)
+      }
+    }
+  }
+
+  override def checkpoint() {
+    // Do nothing. Hadoop RDD should not be checkpointed.
+  }
+
+  override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+    val iter = new Iterator[(K, V)] {
+      var partitionID = "0"
+      val loadMetadataDetails = new LoadMetadataDetails()
+      var model: CarbonLoadModel = _
+      var uniqueLoadStatusId =
+        carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + theSplit.index
+      try {
+        loadMetadataDetails.setPartitionCount(partitionID)
+        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)
+
+        carbonLoadModel.setSegmentId(String.valueOf(loadCount))
+        val recordReaders = getInputIterators
+        val loader = new SparkPartitionLoader(model,
+          theSplit.index,
+          null,
+          null,
+          loadCount,
+          loadMetadataDetails)
+        // Intialize to set carbon properties
+        loader.initialize()
+        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)
+        new DataLoadExecutor().execute(model,
+          loader.storeLocation,
+          recordReaders)
+      } catch {
+        case e: BadRecordFoundException =>
+          loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)
+          logInfo("Bad Record Found")
+        case e: Exception =>
+          logInfo("DataLoad failure", e)
+          LOGGER.error(e)
+          throw e
+      }
+
+      def getInputIterators: Array[util.Iterator[Array[AnyRef]]] = {
+        val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, theSplit.index, 0)
+        val configuration: Configuration = confBroadcast.value.value
+        configureCSVInputFormat(configuration)
+        val hadoopAttemptContext = new TaskAttemptContextImpl(configuration, attemptId)
+        val format = new CSVInputFormat
+        if (isTableSplitPartition) {
+          // for table split partition
+          val split = theSplit.asInstanceOf[CarbonTableSplitPartition]
+          logInfo("Input split: " + split.serializableHadoopSplit.value)
+          carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
+          if (carbonLoadModel.isDirectLoad) {
+            model = carbonLoadModel.getCopyWithPartition(
+                split.serializableHadoopSplit.value.getPartition.getUniqueID,
+                split.serializableHadoopSplit.value.getPartition.getFilesPath,
+                carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
+          } else {
+            model = carbonLoadModel.getCopyWithPartition(
+                split.serializableHadoopSplit.value.getPartition.getUniqueID)
+          }
+          partitionID = split.serializableHadoopSplit.value.getPartition.getUniqueID
+
+          StandardLogService.setThreadName(partitionID, null)
+          CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordPartitionBlockMap(
+              partitionID, split.partitionBlocksDetail.length)
+          val readers =
+          split.partitionBlocksDetail.map(format.createRecordReader(_, hadoopAttemptContext))
+          readers.zipWithIndex.foreach { case (reader, index) =>
+            reader.initialize(split.partitionBlocksDetail(index), hadoopAttemptContext)
+          }
+          readers.map(new RecordReaderIterator(_))
+        } else {
+          // for node partition
+          val split = theSplit.asInstanceOf[CarbonNodePartition]
+          logInfo("Input split: " + split.serializableHadoopSplit)
+          logInfo("The Block Count in this node :" + split.nodeBlocksDetail.length)
+          CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordHostBlockMap(
+              split.serializableHadoopSplit, split.nodeBlocksDetail.length)
+          val blocksID = gernerateBlocksID
+          carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
+          if (carbonLoadModel.isDirectLoad) {
+            val filelist: java.util.List[String] = new java.util.ArrayList[String](
+                CarbonCommonConstants.CONSTANT_SIZE_TEN)
+            CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, filelist, ",")
+            model = carbonLoadModel.getCopyWithPartition(partitionID, filelist,
+                carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
+          } else {
+            model = carbonLoadModel.getCopyWithPartition(partitionID)
+          }
+          StandardLogService.setThreadName(blocksID, null)
+          val readers =
+            split.nodeBlocksDetail.map(format.createRecordReader(_, hadoopAttemptContext))
+          readers.zipWithIndex.foreach { case (reader, index) =>
+            reader.initialize(split.nodeBlocksDetail(index), hadoopAttemptContext)
+          }
+          readers.map(new RecordReaderIterator(_))
+        }
+      }
+
+      def configureCSVInputFormat(configuration: Configuration): Unit = {
+        CSVInputFormat.setCommentCharacter(carbonLoadModel.getCommentChar, configuration)
+        CSVInputFormat.setCSVDelimiter(carbonLoadModel.getCsvDelimiter, configuration)
+        CSVInputFormat.setEscapeCharacter(carbonLoadModel.getEscapeChar, configuration)
+        CSVInputFormat.setHeaderExtractionEnabled(
+          carbonLoadModel.getCsvHeader == null || carbonLoadModel.getCsvHeader.isEmpty,
+          configuration)
+        CSVInputFormat.setQuoteCharacter(carbonLoadModel.getQuoteChar, configuration)
+      }
+
+      /**
+       * generate blocks id
+       *
+       * @return
+       */
+      def gernerateBlocksID: String = {
+        if (isTableSplitPartition) {
+          carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" +
+          theSplit.asInstanceOf[CarbonTableSplitPartition].serializableHadoopSplit.value
+            .getPartition.getUniqueID + "_" + UUID.randomUUID()
+        } else {
+          carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" +
+          UUID.randomUUID()
+        }
+      }
+
+      var finished = false
+
+      override def hasNext: Boolean = {
+        !finished
+      }
+
+      override def next(): (K, V) = {
+        finished = true
+        result.getKey(uniqueLoadStatusId, loadMetadataDetails)
+      }
+    }
+    iter
+  }
+
+  override def getPreferredLocations(split: Partition): Seq[String] = {
+    isTableSplitPartition match {
+      case true =>
+        // for table split partition
+        val theSplit = split.asInstanceOf[CarbonTableSplitPartition]
+        val location = theSplit.serializableHadoopSplit.value.getLocations.asScala
+        location
+      case false =>
+        // for node partition
+        val theSplit = split.asInstanceOf[CarbonNodePartition]
+        val firstOptionLocation: Seq[String] = List(theSplit.serializableHadoopSplit)
+        logInfo("Preferred Location for split : " + firstOptionLocation.head)
+        val blockMap = new util.LinkedHashMap[String, Integer]()
+        val tableBlocks = theSplit.blocksDetails
+        tableBlocks.foreach { tableBlock =>
+          tableBlock.getLocations.foreach { location =>
+            if (!firstOptionLocation.exists(location.equalsIgnoreCase(_))) {
+              val currentCount = blockMap.get(location)
+              if (currentCount == null) {
+                blockMap.put(location, 1)
+              } else {
+                blockMap.put(location, currentCount + 1)
+              }
+            }
+          }
+        }
+
+        val sortedList = blockMap.entrySet().asScala.toSeq.sortWith {(nodeCount1, nodeCount2) =>
+          nodeCount1.getValue > nodeCount2.getValue
+        }
+
+        val sortedNodesList = sortedList.map(nodeCount => nodeCount.getKey).take(2)
+        firstOptionLocation ++ sortedNodesList
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
new file mode 100644
index 0000000..f4c8cd1
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.hive.{CarbonMetaData, DictionaryMap}
+
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonUtil
+
+case class TransformHolder(rdd: Any, mataData: CarbonMetaData)
+
+object CarbonSparkUtil {
+
+  def createSparkMeta(carbonTable: CarbonTable): CarbonMetaData = {
+    val dimensionsAttr = carbonTable.getDimensionByTableName(carbonTable.getFactTableName)
+        .asScala.map(x => x.getColName) // wf : may be problem
+    val measureAttr = carbonTable.getMeasureByTableName(carbonTable.getFactTableName)
+        .asScala.map(x => x.getColName)
+    val dictionary =
+      carbonTable.getDimensionByTableName(carbonTable.getFactTableName).asScala.map { f =>
+        (f.getColName.toLowerCase,
+            f.hasEncoding(Encoding.DICTIONARY) && !f.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
+                !CarbonUtil.hasComplexDataType(f.getDataType))
+      }
+    CarbonMetaData(dimensionsAttr, measureAttr, carbonTable, DictionaryMap(dictionary.toMap))
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/QueryPlanUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/QueryPlanUtil.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/QueryPlanUtil.scala
new file mode 100644
index 0000000..c55c807
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/QueryPlanUtil.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.util
+
+import scala.reflect.ClassTag
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.mapreduce.Job
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
+
+import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier
+import org.apache.carbondata.hadoop.CarbonInputFormat
+
+
+/**
+ * All the utility functions for carbon plan creation
+ */
+object QueryPlanUtil {
+
+  /**
+   * createCarbonInputFormat from query model
+   */
+  def createCarbonInputFormat(absoluteTableIdentifier: AbsoluteTableIdentifier) :
+  (CarbonInputFormat[Array[Object]], Job) = {
+    val carbonInputFormat = new CarbonInputFormat[Array[Object]]()
+    val jobConf: JobConf = new JobConf(new Configuration)
+    val job: Job = new Job(jobConf)
+    FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getTablePath))
+    (carbonInputFormat, job)
+  }
+
+  def createCarbonInputFormat[V: ClassTag](absoluteTableIdentifier: AbsoluteTableIdentifier,
+      conf: Configuration) : CarbonInputFormat[V] = {
+    val carbonInputFormat = new CarbonInputFormat[V]()
+    val job: Job = new Job(conf)
+    FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getTablePath))
+    carbonInputFormat
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala b/integration/spark2/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala
new file mode 100644
index 0000000..cee026d
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.repl
+
+class CarbonSparkILoop extends SparkILoop {
+
+  override def initializeSpark() {
+    intp.beQuietDuring {
+      command("""
+         if(org.apache.spark.repl.carbon.Main.interp == null) {
+           org.apache.spark.repl.carbon.Main.main(Array[String]())
+         }
+              """)
+      command("val i1 = org.apache.spark.repl.carbon.Main.interp")
+      command("import i1._")
+      command("""
+         @transient val sc = {
+           val _sc = i1.createSparkContext()
+           println("Spark context available as sc.")
+           _sc
+         }
+              """)
+      command("import org.apache.spark.SparkContext._")
+      command("import org.apache.spark.sql.CarbonContext")
+      command("""
+         @transient val cc = {
+           val _cc = {
+             import java.io.File
+             val path = System.getenv("CARBON_HOME") + "/bin/carbonshellstore"
+             val store = new File(path)
+             store.mkdirs()
+             val storePath = sc.getConf.getOption("spark.carbon.storepath")
+                  .getOrElse(store.getCanonicalPath)
+             new CarbonContext(sc, storePath, store.getCanonicalPath)
+           }
+           println("Carbon context available as cc.")
+           _cc
+         }
+              """)
+
+      command("import org.apache.spark.sql.SQLContext")
+      command("""
+         @transient val sqlContext = {
+           val _sqlContext = new SQLContext(sc)
+           println("SQL context available as sqlContext.")
+           _sqlContext
+         }
+              """)
+      command("import sqlContext.implicits._")
+      command("import sqlContext.sql")
+
+      command("import cc.implicits._")
+      command("import cc.sql")
+      command("import org.apache.spark.sql.functions._")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
new file mode 100644
index 0000000..c152e0c
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.{TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical.{UnaryNode, _}
+import org.apache.spark.sql.optimizer.{CarbonDecoderRelation}
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.spark.CarbonAliasDecoderRelation
+
+/**
+ * Top command
+ */
+case class Top(count: Int, topOrBottom: Int, dim: NamedExpression, msr: NamedExpression,
+    child: LogicalPlan) extends UnaryNode {
+  def output: Seq[Attribute] = child.output
+
+  override def references: AttributeSet = {
+    val list = List(dim, msr)
+    AttributeSet(list.flatMap(_.references))
+  }
+}
+
+/**
+ * Shows Loads in a table
+ */
+case class ShowLoadsCommand(databaseNameOp: Option[String], table: String, limit: Option[String])
+  extends LogicalPlan with Command {
+
+  override def children: Seq[LogicalPlan] = Seq.empty
+
+  override def output: Seq[Attribute] = {
+    Seq(AttributeReference("SegmentSequenceId", StringType, nullable = false)(),
+      AttributeReference("Status", StringType, nullable = false)(),
+      AttributeReference("Load Start Time", TimestampType, nullable = false)(),
+      AttributeReference("Load End Time", TimestampType, nullable = false)())
+  }
+}
+
+/**
+ * Describe formatted for hive table
+ */
+case class DescribeFormattedCommand(sql: String, tblIdentifier: TableIdentifier)
+  extends LogicalPlan with Command {
+  override def children: Seq[LogicalPlan] = Seq.empty
+
+  override def output: Seq[AttributeReference] =
+    Seq(AttributeReference("result", StringType, nullable = false)())
+}
+
+case class CarbonDictionaryCatalystDecoder(
+    relations: Seq[CarbonDecoderRelation],
+    profile: CarbonProfile,
+    aliasMap: CarbonAliasDecoderRelation,
+    isOuter: Boolean,
+    child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = child.output
+}
+
+abstract class CarbonProfile(attributes: Seq[Attribute]) extends Serializable {
+  def isEmpty: Boolean = attributes.isEmpty
+}
+
+case class IncludeProfile(attributes: Seq[Attribute]) extends CarbonProfile(attributes)
+
+case class ExcludeProfile(attributes: Seq[Attribute]) extends CarbonProfile(attributes)
+
+case class CreateDatabase(dbName: String, sql: String) extends LogicalPlan with Command {
+  override def children: Seq[LogicalPlan] = Seq.empty
+  override def output: Seq[AttributeReference] = {
+    Seq()
+  }
+}
+
+case class DropDatabase(dbName: String, isCascade: Boolean, sql: String)
+    extends LogicalPlan with Command {
+  override def children: Seq[LogicalPlan] = Seq.empty
+  override def output: Seq[AttributeReference] = {
+    Seq()
+  }
+}


[2/6] incubator-carbondata git commit: add spark2 module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
new file mode 100644
index 0000000..794de02
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -0,0 +1,750 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.command
+
+import java.text.SimpleDateFormat
+
+import scala.collection.JavaConverters._
+import scala.language.implicitConversions
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Cast, Literal}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.types.TimestampType
+import org.apache.spark.util.FileUtils
+import org.codehaus.jackson.map.ObjectMapper
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.carbon.CarbonDataLoadSchema
+import org.apache.carbondata.core.carbon.metadata.CarbonMetadata
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding
+import org.apache.carbondata.core.carbon.metadata.schema.table.{CarbonTable, TableInfo}
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.lcm.locks.{CarbonLockFactory, LockUsage}
+import org.apache.carbondata.lcm.status.SegmentStatusManager
+import org.apache.carbondata.processing.constants.TableOptionConstant
+import org.apache.carbondata.processing.etl.DataLoadingException
+import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil}
+
+/**
+ * Command for the compaction in alter table command
+ *
+ * @param alterTableModel
+ */
+case class AlterTableCompaction(alterTableModel: AlterTableModel) extends
+  RunnableCommand {
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+    // TODO : Implement it.
+    val tableName = alterTableModel.tableName
+    val databaseName = alterTableModel.dbName.getOrElse(sparkSession.catalog.currentDatabase)
+    if (null == org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
+      .getCarbonTable(databaseName + "_" + tableName)) {
+      logError(s"alter table failed. table not found: $databaseName.$tableName")
+      sys.error(s"alter table failed. table not found: $databaseName.$tableName")
+    }
+
+    val relation =
+      CarbonEnv.get.carbonMetastore
+        .lookupRelation(Option(databaseName), tableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      sys.error(s"Table $databaseName.$tableName does not exist")
+    }
+    val carbonLoadModel = new CarbonLoadModel()
+
+
+    val table = relation.tableMeta.carbonTable
+    carbonLoadModel.setAggTables(table.getAggregateTablesName.asScala.toArray)
+    carbonLoadModel.setTableName(table.getFactTableName)
+    val dataLoadSchema = new CarbonDataLoadSchema(table)
+    // Need to fill dimension relation
+    carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+    carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
+    carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
+    carbonLoadModel.setStorePath(relation.tableMeta.storePath)
+
+    val kettleHomePath = CarbonScalaUtil.getKettleHome(sparkSession.sqlContext)
+
+    var storeLocation = CarbonProperties.getInstance
+      .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
+        System.getProperty("java.io.tmpdir")
+      )
+    storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
+    try {
+      CarbonDataRDDFactory
+        .alterTableForCompaction(sparkSession.sqlContext,
+          alterTableModel,
+          carbonLoadModel,
+          relation.tableMeta.storePath,
+          kettleHomePath,
+          storeLocation
+        )
+    } catch {
+      case e: Exception =>
+        if (null != e.getMessage) {
+          sys.error(s"Compaction failed. Please check logs for more info. ${ e.getMessage }")
+        } else {
+          sys.error("Exception in compaction. Please check logs for more info.")
+        }
+    }
+    Seq.empty
+  }
+}
+
+case class CreateTable(cm: TableModel) extends RunnableCommand {
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    cm.databaseName = cm.databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+    val tbName = cm.tableName
+    val dbName = cm.databaseName
+    LOGGER.audit(s"Creating Table with Database name [$dbName] and Table name [$tbName]")
+
+    val tableInfo: TableInfo = TableNewProcessor(cm, sparkSession.sqlContext)
+
+    if (tableInfo.getFactTable.getListOfColumns.size <= 0) {
+      sys.error("No Dimensions found. Table should have at least one dimesnion !")
+    }
+
+//    if (sparkSession.sqlContext.tableNames(dbName).exists(_.equalsIgnoreCase(tbName))) {
+//      if (!cm.ifNotExistsSet) {
+//        LOGGER.audit(
+//          s"Table creation with Database name [$dbName] and Table name [$tbName] failed. " +
+//          s"Table [$tbName] already exists under database [$dbName]")
+//        sys.error(s"Table [$tbName] already exists under database [$dbName]")
+//      }
+//    } else {
+      // Add Database to catalog and persist
+      val catalog = CarbonEnv.get.carbonMetastore
+      val tablePath = catalog.createTableFromThrift(tableInfo, dbName, tbName)(sparkSession)
+//      try {
+//        sparkSession.sql(
+//          s"""CREATE TABLE $dbName.$tbName USING carbondata""" +
+//          s""" OPTIONS (tableName "$dbName.$tbName", tablePath "$tablePath") """)
+//          .collect
+//      } catch {
+//        case e: Exception =>
+//          val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
+//          // call the drop table to delete the created table.
+//
+//          CarbonEnv.get.carbonMetastore
+//            .dropTable(catalog.storePath, identifier)(sparkSession)
+//
+//          LOGGER.audit(s"Table creation with Database name [$dbName] " +
+//                       s"and Table name [$tbName] failed")
+//          throw e
+//      }
+
+      LOGGER.audit(s"Table created with Database name [$dbName] and Table name [$tbName]")
+//    }
+
+    Seq.empty
+  }
+
+  def setV(ref: Any, name: String, value: Any): Unit = {
+    ref.getClass.getFields.find(_.getName == name).get
+      .set(ref, value.asInstanceOf[AnyRef])
+  }
+}
+
+case class DeleteLoadsById(
+    loadids: Seq[String],
+    databaseNameOp: Option[String],
+    tableName: String) extends RunnableCommand {
+
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+
+    val databaseName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Delete segment by Id request has been received for $databaseName.$tableName")
+
+    // validate load ids first
+    validateLoadIds
+    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    val relation = CarbonEnv.get.carbonMetastore.lookupRelation(
+      identifier, None)(sparkSession).asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      LOGGER.audit(s"Delete segment by Id is failed. Table $dbName.$tableName does not exist")
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(dbName + '_' + tableName)
+
+    if (null == carbonTable) {
+      CarbonEnv.get.carbonMetastore
+        .lookupRelation(identifier, None)(sparkSession).asInstanceOf[CarbonRelation]
+    }
+    val path = carbonTable.getMetaDataFilepath
+
+    try {
+      val invalidLoadIds = SegmentStatusManager.updateDeletionStatus(
+        carbonTable.getAbsoluteTableIdentifier, loadids.asJava, path).asScala
+
+      if (invalidLoadIds.isEmpty) {
+
+        LOGGER.audit(s"Delete segment by Id is successfull for $databaseName.$tableName.")
+      }
+      else {
+        sys.error("Delete segment by Id is failed. Invalid ID is:" +
+                  s" ${ invalidLoadIds.mkString(",") }")
+      }
+    } catch {
+      case ex: Exception =>
+        sys.error(ex.getMessage)
+    }
+
+    Seq.empty
+
+  }
+
+  // validates load ids
+  private def validateLoadIds: Unit = {
+    if (loadids.isEmpty) {
+      val errorMessage = "Error: Segment id(s) should not be empty."
+      throw new MalformedCarbonCommandException(errorMessage)
+
+    }
+  }
+}
+
+case class DeleteLoadsByLoadDate(
+    databaseNameOp: Option[String],
+    tableName: String,
+    dateField: String,
+    loadDate: String) extends RunnableCommand {
+
+  val LOGGER = LogServiceFactory.getLogService("org.apache.spark.sql.TableModel.tableSchema")
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+
+    LOGGER.audit("The delete segment by load date request has been received.")
+    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    val relation = CarbonEnv.get.carbonMetastore
+      .lookupRelation(identifier, None)(sparkSession).asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      LOGGER
+        .audit(s"Delete segment by load date is failed. Table $dbName.$tableName does not " +
+               s"exist")
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+
+    val timeObj = Cast(Literal(loadDate), TimestampType).eval()
+    if (null == timeObj) {
+      val errorMessage = "Error: Invalid load start time format " + loadDate
+      throw new MalformedCarbonCommandException(errorMessage)
+    }
+
+    val carbonTable = org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
+      .getCarbonTable(dbName + '_' + tableName)
+
+    if (null == carbonTable) {
+      var relation = CarbonEnv.get.carbonMetastore
+        .lookupRelation(identifier, None)(sparkSession).asInstanceOf[CarbonRelation]
+    }
+    val path = carbonTable.getMetaDataFilepath()
+
+    try {
+      val invalidLoadTimestamps = SegmentStatusManager.updateDeletionStatus(
+        carbonTable.getAbsoluteTableIdentifier, loadDate, path,
+        timeObj.asInstanceOf[java.lang.Long]).asScala
+      if (invalidLoadTimestamps.isEmpty) {
+        LOGGER.audit(s"Delete segment by date is successfull for $dbName.$tableName.")
+      }
+      else {
+        sys.error("Delete segment by date is failed. No matching segment found.")
+      }
+    } catch {
+      case ex: Exception =>
+        sys.error(ex.getMessage)
+    }
+    Seq.empty
+
+  }
+
+}
+
+case class LoadTable(
+    databaseNameOp: Option[String],
+    tableName: String,
+    factPathFromUser: String,
+    dimFilesPath: Seq[DataLoadTableFileMapping],
+    options: scala.collection.immutable.Map[String, String],
+    isOverwriteExist: Boolean = false,
+    var inputSqlString: String = null,
+    dataFrame: Option[DataFrame] = None) extends RunnableCommand {
+
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+
+    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    if (isOverwriteExist) {
+      sys.error(s"Overwrite is not supported for carbon table with $dbName.$tableName")
+    }
+    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
+      logError(s"Data loading failed. table not found: $dbName.$tableName")
+      LOGGER.audit(s"Data loading failed. table not found: $dbName.$tableName")
+      sys.error(s"Data loading failed. table not found: $dbName.$tableName")
+    }
+
+    val relation = CarbonEnv.get.carbonMetastore
+        .lookupRelation(Option(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+    CarbonProperties.getInstance().addProperty("zookeeper.enable.lock", "false")
+    val carbonLock = CarbonLockFactory
+      .getCarbonLockObj(relation.tableMeta.carbonTable.getAbsoluteTableIdentifier
+        .getCarbonTableIdentifier,
+        LockUsage.METADATA_LOCK
+      )
+    try {
+      if (carbonLock.lockWithRetries()) {
+        logInfo("Successfully able to get the table metadata file lock")
+      } else {
+        sys.error("Table is locked for updation. Please try after some time")
+      }
+
+      val factPath = if (dataFrame.isDefined) {
+        ""
+      } else {
+        FileUtils.getPaths(
+          CarbonUtil.checkAndAppendHDFSUrl(factPathFromUser))
+      }
+      val carbonLoadModel = new CarbonLoadModel()
+      carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
+      carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
+      carbonLoadModel.setStorePath(relation.tableMeta.storePath)
+      if (dimFilesPath.isEmpty) {
+        carbonLoadModel.setDimFolderPath(null)
+      } else {
+        val x = dimFilesPath.map(f => f.table + ":" + CarbonUtil.checkAndAppendHDFSUrl(f.loadPath))
+        carbonLoadModel.setDimFolderPath(x.mkString(","))
+      }
+
+      val table = relation.tableMeta.carbonTable
+      carbonLoadModel.setAggTables(table.getAggregateTablesName.asScala.toArray)
+      carbonLoadModel.setTableName(table.getFactTableName)
+      val dataLoadSchema = new CarbonDataLoadSchema(table)
+      // Need to fill dimension relation
+      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+
+      var partitionLocation = relation.tableMeta.storePath + "/partition/" +
+                              relation.tableMeta.carbonTableIdentifier.getDatabaseName + "/" +
+                              relation.tableMeta.carbonTableIdentifier.getTableName + "/"
+
+
+      val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean
+      val kettleHomePath = CarbonScalaUtil.getKettleHome(sparkSession.sqlContext)
+
+      // TODO It will be removed after kettle is removed.
+      val useKettle = options.get("use_kettle") match {
+        case Some(value) => value.toBoolean
+        case _ =>
+          val useKettleLocal = System.getProperty("use.kettle")
+          if (useKettleLocal == null) {
+            sparkSession.sqlContext.sparkContext.getConf.get("use_kettle_default", "true").toBoolean
+          } else {
+            useKettleLocal.toBoolean
+          }
+      }
+
+      val delimiter = options.getOrElse("delimiter", ",")
+      val quoteChar = options.getOrElse("quotechar", "\"")
+      val fileHeader = options.getOrElse("fileheader", "")
+      val escapeChar = options.getOrElse("escapechar", "\\")
+      val commentchar = options.getOrElse("commentchar", "#")
+      val columnDict = options.getOrElse("columndict", null)
+      val serializationNullFormat = options.getOrElse("serialization_null_format", "\\N")
+      val badRecordsLoggerEnable = options.getOrElse("bad_records_logger_enable", "false")
+      val badRecordsLoggerRedirect = options.getOrElse("bad_records_action", "force")
+      val allDictionaryPath = options.getOrElse("all_dictionary_path", "")
+      val complex_delimiter_level_1 = options.getOrElse("complex_delimiter_level_1", "\\$")
+      val complex_delimiter_level_2 = options.getOrElse("complex_delimiter_level_2", "\\:")
+      val dateFormat = options.getOrElse("dateformat", null)
+      validateDateFormat(dateFormat, table)
+      val multiLine = options.getOrElse("multiline", "false").trim.toLowerCase match {
+        case "true" => true
+        case "false" => false
+        case illegal =>
+          val errorMessage = "Illegal syntax found: [" + illegal + "] .The value multiline in " +
+                             "load DDL which you set can only be 'true' or 'false', please check " +
+                             "your input DDL."
+          throw new MalformedCarbonCommandException(errorMessage)
+      }
+      val maxColumns = options.getOrElse("maxcolumns", null)
+      carbonLoadModel.setMaxColumns(maxColumns)
+      carbonLoadModel.setEscapeChar(escapeChar)
+      carbonLoadModel.setQuoteChar(quoteChar)
+      carbonLoadModel.setCommentChar(commentchar)
+      carbonLoadModel.setDateFormat(dateFormat)
+      carbonLoadModel
+        .setSerializationNullFormat(
+          TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + "," + serializationNullFormat)
+      carbonLoadModel
+        .setBadRecordsLoggerEnable(
+          TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName + "," + badRecordsLoggerEnable)
+      carbonLoadModel
+        .setBadRecordsAction(
+          TableOptionConstant.BAD_RECORDS_ACTION.getName + "," + badRecordsLoggerRedirect)
+
+      if (delimiter.equalsIgnoreCase(complex_delimiter_level_1) ||
+          complex_delimiter_level_1.equalsIgnoreCase(complex_delimiter_level_2) ||
+          delimiter.equalsIgnoreCase(complex_delimiter_level_2)) {
+        sys.error(s"Field Delimiter & Complex types delimiter are same")
+      }
+      else {
+        carbonLoadModel.setComplexDelimiterLevel1(
+          CarbonUtil.delimiterConverter(complex_delimiter_level_1))
+        carbonLoadModel.setComplexDelimiterLevel2(
+          CarbonUtil.delimiterConverter(complex_delimiter_level_2))
+      }
+      // set local dictionary path, and dictionary file extension
+      carbonLoadModel.setAllDictPath(allDictionaryPath)
+
+      var partitionStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
+      try {
+        // First system has to partition the data first and then call the load data
+        LOGGER.info(s"Initiating Direct Load for the Table : ($dbName.$tableName)")
+        carbonLoadModel.setFactFilePath(factPath)
+        carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimiter))
+        carbonLoadModel.setCsvHeader(fileHeader)
+        carbonLoadModel.setColDictFilePath(columnDict)
+        carbonLoadModel.setDirectLoad(true)
+        GlobalDictionaryUtil
+          .generateGlobalDictionary(
+          sparkSession.sqlContext, carbonLoadModel, relation.tableMeta.storePath, dataFrame)
+        CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+            carbonLoadModel,
+            relation.tableMeta.storePath,
+            kettleHomePath,
+            columnar,
+            partitionStatus,
+            useKettle,
+            dataFrame)
+      } catch {
+        case ex: Exception =>
+          LOGGER.error(ex)
+          LOGGER.audit(s"Dataload failure for $dbName.$tableName. Please check the logs")
+          throw ex
+      } finally {
+        // Once the data load is successful delete the unwanted partition files
+        try {
+          val fileType = FileFactory.getFileType(partitionLocation)
+          if (FileFactory.isFileExist(partitionLocation, fileType)) {
+            val file = FileFactory
+              .getCarbonFile(partitionLocation, fileType)
+            CarbonUtil.deleteFoldersAndFiles(file)
+          }
+        } catch {
+          case ex: Exception =>
+            LOGGER.error(ex)
+            LOGGER.audit(s"Dataload failure for $dbName.$tableName. " +
+                         "Problem deleting the partition folder")
+            throw ex
+        }
+
+      }
+    } catch {
+      case dle: DataLoadingException =>
+        LOGGER.audit(s"Dataload failed for $dbName.$tableName. " + dle.getMessage)
+        throw dle
+      case mce: MalformedCarbonCommandException =>
+        LOGGER.audit(s"Dataload failed for $dbName.$tableName. " + mce.getMessage)
+        throw mce
+    } finally {
+      if (carbonLock != null) {
+        if (carbonLock.unlock()) {
+          logInfo("Table MetaData Unlocked Successfully after data load")
+        } else {
+          logError("Unable to unlock Table MetaData")
+        }
+      }
+    }
+    Seq.empty
+  }
+
+  private def validateDateFormat(dateFormat: String, table: CarbonTable): Unit = {
+    val dimensions = table.getDimensionByTableName(tableName).asScala
+    if (dateFormat != null) {
+      if (dateFormat.trim == "") {
+        throw new MalformedCarbonCommandException("Error: Option DateFormat is set an empty " +
+                                                  "string.")
+      } else {
+        var dateFormats: Array[String] = dateFormat.split(CarbonCommonConstants.COMMA)
+        for (singleDateFormat <- dateFormats) {
+          val dateFormatSplits: Array[String] = singleDateFormat.split(":", 2)
+          val columnName = dateFormatSplits(0).trim.toLowerCase
+          if (!dimensions.exists(_.getColName.equals(columnName))) {
+            throw new MalformedCarbonCommandException("Error: Wrong Column Name " +
+                                                      dateFormatSplits(0) +
+                                                      " is provided in Option DateFormat.")
+          }
+          if (dateFormatSplits.length < 2 || dateFormatSplits(1).trim.isEmpty) {
+            throw new MalformedCarbonCommandException("Error: Option DateFormat is not provided " +
+                                                      "for " + "Column " + dateFormatSplits(0) +
+                                                      ".")
+          }
+        }
+      }
+    }
+  }
+}
+
+private[sql] case class DescribeCommandFormatted(
+    child: SparkPlan,
+    override val output: Seq[Attribute],
+    tblIdentifier: TableIdentifier)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val relation = CarbonEnv.get.carbonMetastore
+      .lookupRelation(tblIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
+    val mapper = new ObjectMapper()
+    val colProps = StringBuilder.newBuilder
+    var results: Seq[(String, String, String)] = child.schema.fields.map { field =>
+      val comment = if (relation.metaData.dims.contains(field.name)) {
+        val dimension = relation.metaData.carbonTable.getDimensionByName(
+          relation.tableMeta.carbonTableIdentifier.getTableName,
+          field.name)
+        if (null != dimension.getColumnProperties && dimension.getColumnProperties.size() > 0) {
+          val colprop = mapper.writeValueAsString(dimension.getColumnProperties)
+          colProps.append(field.name).append(".")
+            .append(mapper.writeValueAsString(dimension.getColumnProperties))
+            .append(",")
+        }
+        if (dimension.hasEncoding(Encoding.DICTIONARY) &&
+            !dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          "DICTIONARY, KEY COLUMN"
+        } else {
+          "KEY COLUMN"
+        }
+      } else {
+        ("MEASURE")
+      }
+      (field.name, field.dataType.simpleString, comment)
+    }
+    val colPropStr = if (colProps.toString().trim().length() > 0) {
+      // drops additional comma at end
+      colProps.toString().dropRight(1)
+    } else {
+      colProps.toString()
+    }
+    results ++= Seq(("", "", ""), ("##Detailed Table Information", "", ""))
+    results ++= Seq(("Database Name: ", relation.tableMeta.carbonTableIdentifier
+      .getDatabaseName, "")
+    )
+    results ++= Seq(("Table Name: ", relation.tableMeta.carbonTableIdentifier.getTableName, ""))
+    results ++= Seq(("CARBON Store Path: ", relation.tableMeta.storePath, ""))
+    val carbonTable = relation.tableMeta.carbonTable
+    results ++= Seq(("Table Block Size : ", carbonTable.getBlockSizeInMB + " MB", ""))
+    results ++= Seq(("", "", ""), ("##Detailed Column property", "", ""))
+    if (colPropStr.length() > 0) {
+      results ++= Seq((colPropStr, "", ""))
+    } else {
+      results ++= Seq(("NONE", "", ""))
+    }
+    val dimension = carbonTable
+      .getDimensionByTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
+    results ++= getColumnGroups(dimension.asScala.toList)
+    results.map { case (name, dataType, comment) =>
+      Row(f"$name%-36s $dataType%-80s $comment%-72s")
+    }
+  }
+
+  private def getColumnGroups(dimensions: List[CarbonDimension]): Seq[(String, String, String)] = {
+    var results: Seq[(String, String, String)] =
+      Seq(("", "", ""), ("##Column Group Information", "", ""))
+    val groupedDimensions = dimensions.groupBy(x => x.columnGroupId()).filter {
+      case (groupId, _) => groupId != -1
+    }.toSeq.sortBy(_._1)
+    val groups = groupedDimensions.map(colGroups => {
+      colGroups._2.map(dim => dim.getColName).mkString(", ")
+    })
+    var index = 1
+    groups.map { x =>
+      results = results :+ (s"Column Group $index", x, "")
+      index = index + 1
+    }
+    results
+  }
+}
+
+private[sql] case class DeleteLoadByDate(
+    databaseNameOp: Option[String],
+    tableName: String,
+    dateField: String,
+    dateValue: String
+) extends RunnableCommand {
+
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"The delete load by date request has been received for $dbName.$tableName")
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    val relation = CarbonEnv.get.carbonMetastore
+      .lookupRelation(identifier)(sparkSession).asInstanceOf[CarbonRelation]
+    var level: String = ""
+    val carbonTable = org.apache.carbondata.core.carbon.metadata.CarbonMetadata
+      .getInstance().getCarbonTable(dbName + '_' + tableName)
+    if (relation == null) {
+      LOGGER.audit(s"The delete load by date is failed. Table $dbName.$tableName does not exist")
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+    val matches: Seq[AttributeReference] = relation.dimensionsAttr.filter(
+      filter => filter.name.equalsIgnoreCase(dateField) &&
+                filter.dataType.isInstanceOf[TimestampType]).toList
+    if (matches.isEmpty) {
+      LOGGER.audit("The delete load by date is failed. " +
+                   s"Table $dbName.$tableName does not contain date field: $dateField")
+      sys.error(s"Table $dbName.$tableName does not contain date field $dateField")
+    } else {
+      level = matches.asJava.get(0).name
+    }
+    val actualColName = relation.metaData.carbonTable.getDimensionByName(tableName, level)
+      .getColName
+    CarbonDataRDDFactory.deleteLoadByDate(
+      sparkSession.sqlContext,
+      new CarbonDataLoadSchema(carbonTable),
+      dbName,
+      tableName,
+      CarbonEnv.get.carbonMetastore.storePath,
+      level,
+      actualColName,
+      dateValue)
+    LOGGER.audit(s"The delete load by date $dateValue is successful for $dbName.$tableName.")
+    Seq.empty
+  }
+
+}
+
+case class CleanFiles(
+    databaseNameOp: Option[String],
+    tableName: String) extends RunnableCommand {
+
+  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  def run(sparkSession: SparkSession): Seq[Row] = {
+    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"The clean files request has been received for $dbName.$tableName")
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    val relation = CarbonEnv.get.carbonMetastore
+      .lookupRelation(identifier)(sparkSession).asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      LOGGER.audit(s"The clean files request is failed. Table $dbName.$tableName does not exist")
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+
+    val carbonLoadModel = new CarbonLoadModel()
+    carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
+    carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
+    val table = relation.tableMeta.carbonTable
+    carbonLoadModel.setAggTables(table.getAggregateTablesName.asScala.toArray)
+    carbonLoadModel.setTableName(table.getFactTableName)
+    carbonLoadModel.setStorePath(relation.tableMeta.storePath)
+    val dataLoadSchema = new CarbonDataLoadSchema(table)
+    carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+    try {
+      CarbonDataRDDFactory.cleanFiles(
+        sparkSession.sqlContext.sparkContext,
+        carbonLoadModel,
+        relation.tableMeta.storePath)
+      LOGGER.audit(s"Clean files request is successfull for $dbName.$tableName.")
+    } catch {
+      case ex: Exception =>
+        sys.error(ex.getMessage)
+    }
+    Seq.empty
+  }
+}
+
+case class ShowLoads(
+    databaseNameOp: Option[String],
+    tableName: String,
+    limit: Option[String],
+    override val output: Seq[Attribute]) extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val databaseName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+    val tableUniqueName = databaseName + "_" + tableName
+    // Here using checkSchemasModifiedTimeAndReloadTables in tableExists to reload metadata if
+    // schema is changed by other process, so that tableInfoMap woulb be refilled.
+    val tableExists = CarbonEnv.get.carbonMetastore
+        .tableExists(TableIdentifier(tableName, databaseNameOp))(sparkSession)
+    if (!tableExists) {
+      sys.error(s"$databaseName.$tableName is not found")
+    }
+    val carbonTable = org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
+        .getCarbonTable(tableUniqueName)
+    if (carbonTable == null) {
+      sys.error(s"$databaseName.$tableName is not found")
+    }
+    val path = carbonTable.getMetaDataFilepath
+    val loadMetadataDetailsArray = SegmentStatusManager.readLoadMetadata(path)
+    if (loadMetadataDetailsArray.nonEmpty) {
+
+      val parser = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP)
+
+      var loadMetadataDetailsSortedArray = loadMetadataDetailsArray.sortWith(
+        (l1, l2) => java.lang.Double.parseDouble(l1.getLoadName) > java.lang.Double
+            .parseDouble(l2.getLoadName)
+      )
+
+
+      if (limit.isDefined) {
+        loadMetadataDetailsSortedArray = loadMetadataDetailsSortedArray
+            .filter(load => load.getVisibility.equalsIgnoreCase("true"))
+        val limitLoads = limit.get
+        try {
+          val lim = Integer.parseInt(limitLoads)
+          loadMetadataDetailsSortedArray = loadMetadataDetailsSortedArray.slice(0, lim)
+        } catch {
+          case ex: NumberFormatException => sys.error(s" Entered limit is not a valid Number")
+        }
+
+      }
+
+      loadMetadataDetailsSortedArray.filter(load => load.getVisibility.equalsIgnoreCase("true"))
+          .map(load =>
+            Row(
+              load.getLoadName,
+              load.getLoadStatus,
+              new java.sql.Timestamp(parser.parse(load.getLoadStartTime).getTime),
+              new java.sql.Timestamp(parser.parse(load.getTimestamp).getTime))).toSeq
+    } else {
+      Seq.empty
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala
new file mode 100644
index 0000000..86c55d3
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetadataUtil.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.hive
+
+import org.apache.spark.sql.SparkSession
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+
+/**
+ * This class contains all carbon hive metadata related utilities
+ */
+object CarbonHiveMetadataUtil {
+
+  @transient
+  val LOGGER = LogServiceFactory.getLogService(CarbonHiveMetadataUtil.getClass.getName)
+
+
+  /**
+   * This method invalidates the table from HiveMetastoreCatalog before dropping table
+   *
+   * @param schemaName
+   * @param cubeName
+   * @param sparkSession
+   */
+  def invalidateAndDropTable(schemaName: String,
+      cubeName: String,
+      sparkSession: SparkSession): Unit = {
+    val tableWithDb = schemaName + "." + cubeName
+    val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableWithDb)
+    try {
+      // todo(wf): in spark no invalidate method now
+      sparkSession.sql(s"DROP TABLE IF EXISTS $schemaName.$cubeName")
+    } catch {
+      case e: Exception =>
+        LOGGER.audit(
+          s"Error While deleting the table $schemaName.$cubeName during drop carbon table" +
+          e.getMessage)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
new file mode 100644
index 0000000..98b481e
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
@@ -0,0 +1,803 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.hive
+
+import java.io._
+import java.util.{GregorianCalendar, LinkedHashSet, UUID}
+import java.util.concurrent.atomic.AtomicLong
+
+import scala.Array.canBuildFrom
+import scala.collection.mutable.ArrayBuffer
+import scala.collection.JavaConverters._
+import scala.util.parsing.combinator.RegexParsers
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{RuntimeConfig, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NoSuchTableException}
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
+import org.apache.spark.sql.execution.command.Partitioner
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.carbon.CarbonTableIdentifier
+import org.apache.carbondata.core.carbon.metadata.CarbonMetadata
+import org.apache.carbondata.core.carbon.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.carbon.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastorage.store.filesystem.CarbonFile
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory.FileType
+import org.apache.carbondata.core.reader.ThriftReader
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, CarbonUtil}
+import org.apache.carbondata.core.writer.ThriftWriter
+import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
+import org.apache.carbondata.lcm.locks.ZookeeperInit
+import org.apache.carbondata.lcm.status.SegmentStatusManager
+import org.apache.carbondata.spark.merger.TableMeta
+import org.apache.carbondata.spark.util.CarbonSparkUtil
+
+case class MetaData(var tablesMeta: ArrayBuffer[TableMeta])
+
+case class CarbonMetaData(dims: Seq[String],
+    msrs: Seq[String],
+    carbonTable: CarbonTable,
+    dictionaryMap: DictionaryMap)
+
+object CarbonMetastore {
+
+  def readSchemaFileToThriftTable(schemaFilePath: String): TableInfo = {
+    val createTBase = new ThriftReader.TBaseCreator() {
+      override def create(): org.apache.thrift.TBase[TableInfo, TableInfo._Fields] = {
+        new TableInfo()
+      }
+    }
+    val thriftReader = new ThriftReader(schemaFilePath, createTBase)
+    var tableInfo: TableInfo = null
+    try {
+      thriftReader.open()
+      tableInfo = thriftReader.read().asInstanceOf[TableInfo]
+    } finally {
+      thriftReader.close()
+    }
+    tableInfo
+  }
+
+  def writeThriftTableToSchemaFile(schemaFilePath: String, tableInfo: TableInfo): Unit = {
+    val thriftWriter = new ThriftWriter(schemaFilePath, false)
+    try {
+      thriftWriter.open()
+      thriftWriter.write(tableInfo);
+    } finally {
+      thriftWriter.close()
+    }
+  }
+
+}
+
+case class DictionaryMap(dictionaryMap: Map[String, Boolean]) {
+  def get(name: String): Option[Boolean] = {
+    dictionaryMap.get(name.toLowerCase)
+  }
+}
+
+class CarbonMetastore(conf: RuntimeConfig, val storePath: String) extends Logging {
+
+  @transient
+  val LOGGER = LogServiceFactory.getLogService("org.apache.spark.sql.CarbonMetastoreCatalog")
+
+  val tableModifiedTimeStore = new java.util.HashMap[String, Long]()
+  tableModifiedTimeStore
+    .put(CarbonCommonConstants.DATABASE_DEFAULT_NAME, System.currentTimeMillis())
+
+  private val nextId = new AtomicLong(0)
+
+  def nextQueryId: String = {
+    s"query_${nextId.getAndIncrement()}"
+  }
+
+  val metadata = loadMetadata(storePath, nextQueryId)
+
+  def getTableCreationTime(databaseName: String, tableName: String): Long = {
+    val tableMeta = metadata.tablesMeta.filter(
+      c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(databaseName) &&
+           c.carbonTableIdentifier.getTableName.equalsIgnoreCase(tableName))
+    val tableCreationTime = tableMeta.head.carbonTable.getTableLastUpdatedTime
+    tableCreationTime
+  }
+
+  def lookupRelation(dbName: Option[String],
+                     tableName: String)(sparkSession: SparkSession): LogicalPlan = {
+    lookupRelation(TableIdentifier(tableName, dbName))(sparkSession)
+  }
+
+  def lookupRelation(tableIdentifier: TableIdentifier,
+                     alias: Option[String] = None)(sparkSession: SparkSession): LogicalPlan = {
+    checkSchemasModifiedTimeAndReloadTables()
+    val database = tableIdentifier.database.getOrElse(
+      sparkSession.catalog.currentDatabase
+    )
+    val tables = getTableFromMetadata(database, tableIdentifier.table)
+    tables match {
+      case Some(t) =>
+        CarbonRelation(database, tableIdentifier.table,
+          CarbonSparkUtil.createSparkMeta(tables.head.carbonTable), tables.head, alias)
+      case None =>
+        LOGGER.audit(s"Table Not Found: ${tableIdentifier.table}")
+        throw new NoSuchTableException(database, tableIdentifier.table)
+    }
+  }
+
+  /**
+   * This method will search for a table in the catalog metadata
+   *
+   * @param database
+   * @param tableName
+   * @return
+   */
+  def getTableFromMetadata(database: String,
+      tableName: String): Option[TableMeta] = {
+    metadata.tablesMeta
+      .find(c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(database) &&
+                 c.carbonTableIdentifier.getTableName.equalsIgnoreCase(tableName))
+  }
+
+  def tableExists(tableIdentifier: TableIdentifier)(sparkSession: SparkSession): Boolean = {
+    checkSchemasModifiedTimeAndReloadTables()
+    val database = tableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase)
+    val tables = metadata.tablesMeta.filter(
+      c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(database) &&
+           c.carbonTableIdentifier.getTableName.equalsIgnoreCase(tableIdentifier.table))
+    tables.nonEmpty
+  }
+
+  def loadMetadata(metadataPath: String, queryId: String): MetaData = {
+    val recorder = CarbonTimeStatisticsFactory.createDriverRecorder()
+    val statistic = new QueryStatistic()
+    // creating zookeeper instance once.
+    // if zookeeper is configured as carbon lock type.
+    val zookeeperUrl: String = conf.get(CarbonCommonConstants.ZOOKEEPER_URL, null)
+    if (zookeeperUrl != null) {
+      CarbonProperties.getInstance.addProperty(CarbonCommonConstants.ZOOKEEPER_URL, zookeeperUrl)
+      ZookeeperInit.getInstance(zookeeperUrl)
+      LOGGER.info("Zookeeper url is configured. Taking the zookeeper as lock type.")
+      var configuredLockType = CarbonProperties.getInstance
+        .getProperty(CarbonCommonConstants.LOCK_TYPE)
+      if (null == configuredLockType) {
+        configuredLockType = CarbonCommonConstants.CARBON_LOCK_TYPE_ZOOKEEPER
+        CarbonProperties.getInstance
+          .addProperty(CarbonCommonConstants.LOCK_TYPE,
+            configuredLockType)
+      }
+    }
+
+    if (metadataPath == null) {
+      return null
+    }
+    val fileType = FileFactory.getFileType(metadataPath)
+    val metaDataBuffer = new ArrayBuffer[TableMeta]
+    fillMetaData(metadataPath, fileType, metaDataBuffer)
+    updateSchemasUpdatedTime(readSchemaFileSystemTime("", ""))
+    statistic.addStatistics(QueryStatisticsConstants.LOAD_META,
+      System.currentTimeMillis())
+    recorder.recordStatisticsForDriver(statistic, queryId)
+    MetaData(metaDataBuffer)
+  }
+
+  private def fillMetaData(basePath: String, fileType: FileType,
+      metaDataBuffer: ArrayBuffer[TableMeta]): Unit = {
+    val databasePath = basePath // + "/schemas"
+    try {
+      if (FileFactory.isFileExist(databasePath, fileType)) {
+        val file = FileFactory.getCarbonFile(databasePath, fileType)
+        val databaseFolders = file.listFiles()
+
+        databaseFolders.foreach(databaseFolder => {
+          if (databaseFolder.isDirectory) {
+            val dbName = databaseFolder.getName
+            val tableFolders = databaseFolder.listFiles()
+
+            tableFolders.foreach(tableFolder => {
+              if (tableFolder.isDirectory) {
+                val carbonTableIdentifier = new CarbonTableIdentifier(databaseFolder.getName,
+                  tableFolder.getName, UUID.randomUUID().toString)
+                val carbonTablePath = CarbonStorePath.getCarbonTablePath(basePath,
+                  carbonTableIdentifier)
+                val tableMetadataFile = carbonTablePath.getSchemaFilePath
+
+                if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
+                  val tableName = tableFolder.getName
+                  val tableUniqueName = databaseFolder.getName + "_" + tableFolder.getName
+
+
+                  val createTBase = new ThriftReader.TBaseCreator() {
+                    override def create(): org.apache.thrift.TBase[TableInfo, TableInfo._Fields] = {
+                      new TableInfo()
+                    }
+                  }
+                  val thriftReader = new ThriftReader(tableMetadataFile, createTBase)
+                  thriftReader.open()
+                  val tableInfo: TableInfo = thriftReader.read().asInstanceOf[TableInfo]
+                  thriftReader.close()
+
+                  val schemaConverter = new ThriftWrapperSchemaConverterImpl
+                  val wrapperTableInfo = schemaConverter
+                    .fromExternalToWrapperTableInfo(tableInfo, dbName, tableName, basePath)
+                  val schemaFilePath = CarbonStorePath
+                    .getCarbonTablePath(storePath, carbonTableIdentifier).getSchemaFilePath
+                  wrapperTableInfo.setStorePath(storePath)
+                  wrapperTableInfo
+                    .setMetaDataFilepath(CarbonTablePath.getFolderContainingFile(schemaFilePath))
+                  CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo)
+                  val carbonTable =
+                    org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
+                      .getCarbonTable(tableUniqueName)
+                  metaDataBuffer += new TableMeta(carbonTable.getCarbonTableIdentifier, storePath,
+                    carbonTable)
+                }
+              }
+            })
+          }
+        })
+      } else {
+        // Create folders and files.
+        FileFactory.mkdirs(databasePath, fileType)
+      }
+    } catch {
+      case s: java.io.FileNotFoundException =>
+        // Create folders and files.
+        FileFactory.mkdirs(databasePath, fileType)
+    }
+  }
+
+  /**
+   *
+   * Prepare Thrift Schema from wrapper TableInfo and write to Schema file.
+   * Load CarbonTable from wrapper tableinfo
+   *
+   */
+  def createTableFromThrift(
+      tableInfo: org.apache.carbondata.core.carbon.metadata.schema.table.TableInfo,
+      dbName: String, tableName: String)
+    (sparkSession: SparkSession): String = {
+    if (tableExists(TableIdentifier(tableName, Some(dbName)))(sparkSession)) {
+      sys.error(s"Table [$tableName] already exists under Database [$dbName]")
+    }
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl
+    val thriftTableInfo = schemaConverter
+      .fromWrapperToExternalTableInfo(tableInfo, dbName, tableName)
+    val schemaEvolutionEntry = new SchemaEvolutionEntry(tableInfo.getLastUpdatedTime)
+    thriftTableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history
+      .add(schemaEvolutionEntry)
+
+    val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName,
+      tableInfo.getFactTable.getTableId)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
+    val schemaFilePath = carbonTablePath.getSchemaFilePath
+    val schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath)
+    tableInfo.setMetaDataFilepath(schemaMetadataPath)
+    tableInfo.setStorePath(storePath)
+    CarbonMetadata.getInstance().loadTableMetadata(tableInfo)
+    val tableMeta = new TableMeta(carbonTableIdentifier, storePath,
+      CarbonMetadata.getInstance().getCarbonTable(dbName + "_" + tableName))
+
+    val fileType = FileFactory.getFileType(schemaMetadataPath)
+    if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
+      FileFactory.mkdirs(schemaMetadataPath, fileType)
+    }
+    val thriftWriter = new ThriftWriter(schemaFilePath, false)
+    thriftWriter.open()
+    thriftWriter.write(thriftTableInfo)
+    thriftWriter.close()
+    metadata.tablesMeta += tableMeta
+    logInfo(s"Table $tableName for Database $dbName created successfully.")
+    LOGGER.info(s"Table $tableName for Database $dbName created successfully.")
+    updateSchemasUpdatedTime(touchSchemaFileSystemTime(dbName, tableName))
+    carbonTablePath.getPath
+  }
+
+  private def updateMetadataByWrapperTable(
+      wrapperTableInfo: org.apache.carbondata.core.carbon.metadata.schema.table.TableInfo): Unit = {
+
+    CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo)
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(
+      wrapperTableInfo.getTableUniqueName)
+    for (i <- metadata.tablesMeta.indices) {
+      if (wrapperTableInfo.getTableUniqueName.equals(
+        metadata.tablesMeta(i).carbonTableIdentifier.getTableUniqueName)) {
+        metadata.tablesMeta(i).carbonTable = carbonTable
+      }
+    }
+  }
+
+  def updateMetadataByThriftTable(schemaFilePath: String,
+      tableInfo: TableInfo, dbName: String, tableName: String, storePath: String): Unit = {
+
+    tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
+      .setTime_stamp(System.currentTimeMillis())
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl
+    val wrapperTableInfo = schemaConverter
+      .fromExternalToWrapperTableInfo(tableInfo, dbName, tableName, storePath)
+    wrapperTableInfo
+      .setMetaDataFilepath(CarbonTablePath.getFolderContainingFile(schemaFilePath))
+    wrapperTableInfo.setStorePath(storePath)
+    updateMetadataByWrapperTable(wrapperTableInfo)
+  }
+
+  /**
+   * Shows all schemas which has Database name like
+   */
+  def showDatabases(schemaLike: Option[String]): Seq[String] = {
+    checkSchemasModifiedTimeAndReloadTables()
+    metadata.tablesMeta.map { c =>
+      schemaLike match {
+        case Some(name) =>
+          if (c.carbonTableIdentifier.getDatabaseName.contains(name)) {
+            c.carbonTableIdentifier
+              .getDatabaseName
+          } else {
+            null
+          }
+        case _ => c.carbonTableIdentifier.getDatabaseName
+      }
+    }.filter(f => f != null)
+  }
+
+  /**
+   * Shows all tables in all schemas.
+   */
+  def getAllTables(): Seq[TableIdentifier] = {
+    checkSchemasModifiedTimeAndReloadTables()
+    metadata.tablesMeta.map { c =>
+      TableIdentifier(c.carbonTableIdentifier.getTableName,
+        Some(c.carbonTableIdentifier.getDatabaseName))
+    }
+  }
+
+  def isTablePathExists(tableIdentifier: TableIdentifier)(sparkSession: SparkSession): Boolean = {
+    val dbName = tableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase)
+    val tableName = tableIdentifier.table
+
+    val tablePath = CarbonStorePath.getCarbonTablePath(this.storePath,
+      new CarbonTableIdentifier(dbName, tableName, "")).getPath
+
+    val fileType = FileFactory.getFileType(tablePath)
+    FileFactory.isFileExist(tablePath, fileType)
+  }
+
+  def dropTable(tableStorePath: String, tableIdentifier: TableIdentifier)
+    (sparkSession: SparkSession) {
+    val dbName = tableIdentifier.database.get
+    val tableName = tableIdentifier.table
+
+    val metadataFilePath = CarbonStorePath.getCarbonTablePath(tableStorePath,
+      new CarbonTableIdentifier(dbName, tableName, "")).getMetadataDirectoryPath
+
+    val fileType = FileFactory.getFileType(metadataFilePath)
+
+    if (FileFactory.isFileExist(metadataFilePath, fileType)) {
+      // while drop we should refresh the schema modified time so that if any thing has changed
+      // in the other beeline need to update.
+      checkSchemasModifiedTimeAndReloadTables
+      val file = FileFactory.getCarbonFile(metadataFilePath, fileType)
+      CarbonUtil.deleteFoldersAndFilesSilent(file.getParentFile)
+      val metadataToBeRemoved: Option[TableMeta] = getTableFromMetadata(dbName,
+        tableIdentifier.table)
+      metadataToBeRemoved match {
+        case Some(tableMeta) =>
+          metadata.tablesMeta -= tableMeta
+          org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
+            .removeTable(dbName + "_" + tableName)
+          org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
+            .removeTable(dbName + "_" + tableName)
+          updateSchemasUpdatedTime(touchSchemaFileSystemTime(dbName, tableName))
+        case None =>
+          logInfo(s"Metadata does not contain entry for table $tableName in database $dbName")
+      }
+      CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
+      // discard cached table info in cachedDataSourceTables
+      sparkSession.sessionState.catalog.refreshTable(tableIdentifier)
+    }
+  }
+
+  private def getTimestampFileAndType(databaseName: String, tableName: String) = {
+    val timestampFile = storePath + "/" + CarbonCommonConstants.SCHEMAS_MODIFIED_TIME_FILE
+    val timestampFileType = FileFactory.getFileType(timestampFile)
+    (timestampFile, timestampFileType)
+  }
+
+  /**
+   * This method will put the updated timestamp of schema file in the table modified time store map
+   *
+   * @param timeStamp
+   */
+  def updateSchemasUpdatedTime(timeStamp: Long) {
+    tableModifiedTimeStore.put("default", timeStamp)
+  }
+
+  /**
+   * This method will read the timestamp of empty schema file
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  def readSchemaFileSystemTime(databaseName: String, tableName: String): Long = {
+    val (timestampFile, timestampFileType) = getTimestampFileAndType(databaseName, tableName)
+    if (FileFactory.isFileExist(timestampFile, timestampFileType)) {
+      FileFactory.getCarbonFile(timestampFile, timestampFileType).getLastModifiedTime
+    } else {
+      System.currentTimeMillis()
+    }
+  }
+
+  /**
+   * This method will check and create an empty schema timestamp file
+   *
+   * @param databaseName
+   * @param tableName
+   * @return
+   */
+  def touchSchemaFileSystemTime(databaseName: String, tableName: String): Long = {
+    val (timestampFile, timestampFileType) = getTimestampFileAndType(databaseName, tableName)
+    if (!FileFactory.isFileExist(timestampFile, timestampFileType)) {
+      LOGGER.audit(s"Creating timestamp file for $databaseName.$tableName")
+      FileFactory.createNewFile(timestampFile, timestampFileType)
+    }
+    val systemTime = System.currentTimeMillis()
+    FileFactory.getCarbonFile(timestampFile, timestampFileType)
+      .setLastModifiedTime(systemTime)
+    systemTime
+  }
+
+  def checkSchemasModifiedTimeAndReloadTables() {
+    val (timestampFile, timestampFileType) = getTimestampFileAndType("", "")
+    if (FileFactory.isFileExist(timestampFile, timestampFileType)) {
+      if (!(FileFactory.getCarbonFile(timestampFile, timestampFileType).
+        getLastModifiedTime ==
+            tableModifiedTimeStore.get(CarbonCommonConstants.DATABASE_DEFAULT_NAME))) {
+        refreshCache()
+      }
+    }
+  }
+
+  def refreshCache() {
+    metadata.tablesMeta = loadMetadata(storePath, nextQueryId).tablesMeta
+  }
+
+  def getSchemaLastUpdatedTime(databaseName: String, tableName: String): Long = {
+    var schemaLastUpdatedTime = System.currentTimeMillis
+    val (timestampFile, timestampFileType) = getTimestampFileAndType(databaseName, tableName)
+    if (FileFactory.isFileExist(timestampFile, timestampFileType)) {
+      schemaLastUpdatedTime = FileFactory.getCarbonFile(timestampFile, timestampFileType)
+        .getLastModifiedTime
+    }
+    schemaLastUpdatedTime
+  }
+
+  def readTableMetaDataFile(tableFolder: CarbonFile,
+      fileType: FileFactory.FileType):
+  (String, String, String, String, Partitioner, Long) = {
+    val tableMetadataFile = tableFolder.getAbsolutePath + "/metadata"
+
+    var schema: String = ""
+    var databaseName: String = ""
+    var tableName: String = ""
+    var dataPath: String = ""
+    var partitioner: Partitioner = null
+    val cal = new GregorianCalendar(2011, 1, 1)
+    var tableCreationTime = cal.getTime.getTime
+
+    if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
+      // load metadata
+      val in = FileFactory.getDataInputStream(tableMetadataFile, fileType)
+      var len = 0
+      try {
+        len = in.readInt()
+      } catch {
+        case others: EOFException => len = 0
+      }
+
+      while (len > 0) {
+        val databaseNameBytes = new Array[Byte](len)
+        in.readFully(databaseNameBytes)
+
+        databaseName = new String(databaseNameBytes, "UTF8")
+        val tableNameLen = in.readInt()
+        val tableNameBytes = new Array[Byte](tableNameLen)
+        in.readFully(tableNameBytes)
+        tableName = new String(tableNameBytes, "UTF8")
+
+        val dataPathLen = in.readInt()
+        val dataPathBytes = new Array[Byte](dataPathLen)
+        in.readFully(dataPathBytes)
+        dataPath = new String(dataPathBytes, "UTF8")
+
+        val versionLength = in.readInt()
+        val versionBytes = new Array[Byte](versionLength)
+        in.readFully(versionBytes)
+
+        val schemaLen = in.readInt()
+        val schemaBytes = new Array[Byte](schemaLen)
+        in.readFully(schemaBytes)
+        schema = new String(schemaBytes, "UTF8")
+
+        val partitionLength = in.readInt()
+        val partitionBytes = new Array[Byte](partitionLength)
+        in.readFully(partitionBytes)
+        val inStream = new ByteArrayInputStream(partitionBytes)
+        val objStream = new ObjectInputStream(inStream)
+        partitioner = objStream.readObject().asInstanceOf[Partitioner]
+        objStream.close()
+
+        try {
+          tableCreationTime = in.readLong()
+          len = in.readInt()
+        } catch {
+          case others: EOFException => len = 0
+        }
+
+      }
+      in.close()
+    }
+
+    (databaseName, tableName, dataPath, schema, partitioner, tableCreationTime)
+  }
+
+  def createDatabaseDirectory(dbName: String) {
+    val databasePath = storePath + File.separator + dbName
+    val fileType = FileFactory.getFileType(databasePath)
+    FileFactory.mkdirs(databasePath, fileType)
+  }
+
+  def dropDatabaseDirectory(dbName: String) {
+    val databasePath = storePath + File.separator + dbName
+    val fileType = FileFactory.getFileType(databasePath)
+    if (FileFactory.isFileExist(databasePath, fileType)) {
+      val dbPath = FileFactory.getCarbonFile(databasePath, fileType)
+      CarbonUtil.deleteFoldersAndFiles(dbPath)
+    }
+  }
+
+}
+
+
+object CarbonMetastoreTypes extends RegexParsers {
+  protected lazy val primitiveType: Parser[DataType] =
+    "string" ^^^ StringType |
+    "float" ^^^ FloatType |
+    "int" ^^^ IntegerType |
+    "tinyint" ^^^ ShortType |
+    "short" ^^^ ShortType |
+    "double" ^^^ DoubleType |
+    "long" ^^^ LongType |
+    "binary" ^^^ BinaryType |
+    "boolean" ^^^ BooleanType |
+    fixedDecimalType |
+    "decimal" ^^^ "decimal" ^^^ DecimalType(18, 2) |
+    "varchar\\((\\d+)\\)".r ^^^ StringType |
+    "timestamp" ^^^ TimestampType
+
+  protected lazy val fixedDecimalType: Parser[DataType] =
+    "decimal" ~> "(" ~> "^[1-9]\\d*".r ~ ("," ~> "^[0-9]\\d*".r <~ ")") ^^ {
+      case precision ~ scale =>
+        DecimalType(precision.toInt, scale.toInt)
+    }
+
+  protected lazy val arrayType: Parser[DataType] =
+    "array" ~> "<" ~> dataType <~ ">" ^^ {
+      case tpe => ArrayType(tpe)
+    }
+
+  protected lazy val mapType: Parser[DataType] =
+    "map" ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ {
+      case t1 ~ _ ~ t2 => MapType(t1, t2)
+    }
+
+  protected lazy val structField: Parser[StructField] =
+    "[a-zA-Z0-9_]*".r ~ ":" ~ dataType ^^ {
+      case name ~ _ ~ tpe => StructField(name, tpe, nullable = true)
+    }
+
+  protected lazy val structType: Parser[DataType] =
+    "struct" ~> "<" ~> repsep(structField, ",") <~ ">" ^^ {
+      case fields => StructType(fields)
+    }
+
+  protected lazy val dataType: Parser[DataType] =
+    arrayType |
+    mapType |
+    structType |
+    primitiveType
+
+  def toDataType(metastoreType: String): DataType = {
+    parseAll(dataType, metastoreType) match {
+      case Success(result, _) => result
+      case failure: NoSuccess => sys.error(s"Unsupported dataType: $metastoreType")
+    }
+  }
+
+  def toMetastoreType(dt: DataType): String = {
+    dt match {
+      case ArrayType(elementType, _) => s"array<${ toMetastoreType(elementType) }>"
+      case StructType(fields) =>
+        s"struct<${
+          fields.map(f => s"${ f.name }:${ toMetastoreType(f.dataType) }")
+            .mkString(",")
+        }>"
+      case StringType => "string"
+      case FloatType => "float"
+      case IntegerType => "int"
+      case ShortType => "tinyint"
+      case DoubleType => "double"
+      case LongType => "bigint"
+      case BinaryType => "binary"
+      case BooleanType => "boolean"
+      case DecimalType() => "decimal"
+      case TimestampType => "timestamp"
+    }
+  }
+}
+
+
+/**
+ * Represents logical plan for one carbon table
+ */
+case class CarbonRelation(
+    databaseName: String,
+    tableName: String,
+    metaData: CarbonMetaData,
+    tableMeta: TableMeta,
+    alias: Option[String])
+  extends LeafNode with MultiInstanceRelation {
+
+  def recursiveMethod(dimName: String, childDim: CarbonDimension): String = {
+    childDim.getDataType.toString.toLowerCase match {
+      case "array" => s"${
+        childDim.getColName.substring(dimName.length + 1)
+      }:array<${ getArrayChildren(childDim.getColName) }>"
+      case "struct" => s"${
+        childDim.getColName.substring(dimName.length + 1)
+      }:struct<${ getStructChildren(childDim.getColName) }>"
+      case dType => s"${ childDim.getColName.substring(dimName.length + 1) }:${ dType }"
+    }
+  }
+
+  def getArrayChildren(dimName: String): String = {
+    metaData.carbonTable.getChildren(dimName).asScala.map(childDim => {
+      childDim.getDataType.toString.toLowerCase match {
+        case "array" => s"array<${ getArrayChildren(childDim.getColName) }>"
+        case "struct" => s"struct<${ getStructChildren(childDim.getColName) }>"
+        case dType => addDecimalScaleAndPrecision(childDim, dType)
+      }
+    }).mkString(",")
+  }
+
+  def getStructChildren(dimName: String): String = {
+    metaData.carbonTable.getChildren(dimName).asScala.map(childDim => {
+      childDim.getDataType.toString.toLowerCase match {
+        case "array" => s"${
+          childDim.getColName.substring(dimName.length + 1)
+        }:array<${ getArrayChildren(childDim.getColName) }>"
+        case "struct" => s"${
+          childDim.getColName.substring(dimName.length + 1)
+        }:struct<${ metaData.carbonTable.getChildren(childDim.getColName)
+          .asScala.map(f => s"${ recursiveMethod(childDim.getColName, f) }").mkString(",")
+        }>"
+        case dType => s"${ childDim.getColName
+          .substring(dimName.length() + 1) }:${ addDecimalScaleAndPrecision(childDim, dType) }"
+      }
+    }).mkString(",")
+  }
+
+  override def newInstance(): LogicalPlan = {
+    CarbonRelation(databaseName, tableName, metaData, tableMeta, alias)
+      .asInstanceOf[this.type]
+  }
+
+  val dimensionsAttr = {
+    val sett = new LinkedHashSet(
+      tableMeta.carbonTable.getDimensionByTableName(tableMeta.carbonTableIdentifier.getTableName)
+        .asScala.asJava)
+    sett.asScala.toSeq.filter(!_.getColumnSchema.isInvisible).map(dim => {
+      val dimval = metaData.carbonTable
+        .getDimensionByName(metaData.carbonTable.getFactTableName, dim.getColName)
+      val output: DataType = dimval.getDataType
+        .toString.toLowerCase match {
+        case "array" =>
+          CarbonMetastoreTypes.toDataType(s"array<${ getArrayChildren(dim.getColName) }>")
+        case "struct" =>
+          CarbonMetastoreTypes.toDataType(s"struct<${ getStructChildren(dim.getColName) }>")
+        case dType =>
+          val dataType = addDecimalScaleAndPrecision(dimval, dType)
+          CarbonMetastoreTypes.toDataType(dataType)
+      }
+
+      AttributeReference(
+        dim.getColName,
+        output,
+        nullable = true)()
+    })
+  }
+
+  val measureAttr = {
+    val factTable = tableMeta.carbonTable.getFactTableName
+    new LinkedHashSet(
+      tableMeta.carbonTable.
+        getMeasureByTableName(tableMeta.carbonTable.getFactTableName).
+        asScala.asJava).asScala.toSeq.filter(!_.getColumnSchema.isInvisible)
+      .map(x => AttributeReference(x.getColName, CarbonMetastoreTypes.toDataType(
+        metaData.carbonTable.getMeasureByName(factTable, x.getColName).getDataType.toString
+          .toLowerCase match {
+          case "int" => "long"
+          case "short" => "long"
+          case "decimal" => "decimal(" + x.getPrecision + "," + x.getScale + ")"
+          case others => others
+        }),
+        nullable = true)())
+  }
+
+  override val output = dimensionsAttr ++ measureAttr
+
+  // TODO: Use data from the footers.
+  override lazy val statistics = Statistics(sizeInBytes = this.sizeInBytes)
+
+  override def equals(other: Any): Boolean = {
+    other match {
+      case p: CarbonRelation =>
+        p.databaseName == databaseName && p.output == output && p.tableName == tableName
+      case _ => false
+    }
+  }
+
+  def addDecimalScaleAndPrecision(dimval: CarbonDimension, dataType: String): String = {
+    var dType = dataType
+    if (dimval.getDataType
+      == org.apache.carbondata.core.carbon.metadata.datatype.DataType.DECIMAL) {
+      dType +=
+        "(" + dimval.getColumnSchema.getPrecision + "," + dimval.getColumnSchema.getScale + ")"
+    }
+    dType
+  }
+
+  private var tableStatusLastUpdateTime = 0L
+
+  private var sizeInBytesLocalValue = 0L
+
+  def sizeInBytes: Long = {
+    val tableStatusNewLastUpdatedTime = SegmentStatusManager.getTableStatusLastModifiedTime(
+      tableMeta.carbonTable.getAbsoluteTableIdentifier)
+
+    if (tableStatusLastUpdateTime != tableStatusNewLastUpdatedTime) {
+      val tablePath = CarbonStorePath.getCarbonTablePath(
+        tableMeta.storePath,
+        tableMeta.carbonTableIdentifier).getPath
+      val fileType = FileFactory.getFileType(tablePath)
+      if(FileFactory.isFileExist(tablePath, fileType)) {
+        tableStatusLastUpdateTime = tableStatusNewLastUpdatedTime
+        sizeInBytesLocalValue = FileFactory.getDirectorySize(tablePath)
+      }
+    }
+    sizeInBytesLocalValue
+  }
+
+}


[5/6] incubator-carbondata git commit: add spark2 module

Posted by ja...@apache.org.
add spark2 module

rebase

rebase

rebase

rebase

carbon api for spark2

fix late decoder

fix

comment on path

merged caiqiang's fix for decode rule

added test cases

fix style

fix pom

fix 1.5

1.5 example issue

fix 1.5 testsuite

fix 1.5 tests

imports


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

Branch: refs/heads/master
Commit: 590ecceb148dfe19ca3f16766e64dedf3646cf59
Parents: d94b99f
Author: jackylk <ja...@huawei.com>
Authored: Wed Nov 30 16:28:34 2016 +0800
Committer: jackylk <ja...@huawei.com>
Committed: Thu Dec 1 01:45:05 2016 +0800

----------------------------------------------------------------------
 conf/dataload.properties.template               |   73 ++
 core/pom.xml                                    |    7 -
 .../examples/GenerateDictionaryExample.scala    |    7 +-
 examples/spark2/src/main/resources/data.csv     |   11 +
 .../carbondata/examples/CarbonExample.scala     |  109 ++
 hadoop/pom.xml                                  |    7 -
 .../carbondata/hadoop/CarbonProjection.java     |    5 +-
 .../AbstractDictionaryDecodedReadSupport.java   |    2 +
 integration-testcases/pom.xml                   |    8 -
 integration/spark-common/pom.xml                |   10 +-
 .../spark/merger/CarbonCompactionUtil.java      |    1 -
 .../readsupport/SparkRowReadSupportImpl.java    |    9 +-
 .../spark/rdd/CarbonDataLoadRDD.scala           |    3 +-
 .../spark/rdd/DataLoadPartitionCoalescer.scala  |    2 +-
 .../carbondata/spark/util/CommonUtil.scala      |    3 +-
 .../spark/util/GlobalDictionaryUtil.scala       |    2 +-
 .../CarbonTableIdentifierImplicit.scala         |    2 -
 .../spark/sql/hive/DistributionUtil.scala       |   17 +-
 integration/spark/pom.xml                       |    7 -
 .../spark/sql/CarbonDatasourceRelation.scala    |    1 +
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   10 +-
 .../scala/org/apache/spark/sql/CarbonScan.scala |    1 -
 .../org/apache/spark/sql/CarbonSparkUtil.scala  |    3 +-
 integration/spark2/pom.xml                      |    5 +
 .../spark/load/DeleteLoadFromMetadata.java      |   44 +
 .../carbondata/spark/util/CarbonQueryUtil.java  |  248 ++++
 .../spark/CarbonColumnValidator.scala           |   36 +
 .../apache/carbondata/spark/CarbonFilters.scala |  391 ++++++
 .../apache/carbondata/spark/CarbonOption.scala  |   48 +
 .../carbondata/spark/CarbonSparkFactory.scala   |   60 +
 .../spark/DictionaryDetailHelper.scala          |   62 +
 .../org/apache/carbondata/spark/KeyVal.scala    |   89 ++
 .../spark/rdd/CarbonDataRDDFactory.scala        | 1115 ++++++++++++++++++
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  250 ++++
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  305 +++++
 .../carbondata/spark/util/CarbonSparkUtil.scala |   45 +
 .../carbondata/spark/util/QueryPlanUtil.scala   |   56 +
 .../apache/spark/repl/CarbonSparkILoop.scala    |   72 ++
 .../spark/sql/CarbonCatalystOperators.scala     |   98 ++
 .../spark/sql/CarbonDataFrameWriter.scala       |  168 +++
 .../sql/CarbonDatasourceHadoopRelation.scala    |   78 ++
 .../spark/sql/CarbonDictionaryDecoder.scala     |  222 ++++
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   59 +
 .../scala/org/apache/spark/sql/CarbonScan.scala |   11 +-
 .../org/apache/spark/sql/CarbonSource.scala     |  143 +++
 .../spark/sql/SparkUnknownExpression.scala      |  130 ++
 .../org/apache/spark/sql/TableCreator.scala     |  490 ++++++++
 .../execution/CarbonLateDecodeStrategy.scala    |  128 +-
 .../execution/command/carbonTableSchema.scala   |  750 ++++++++++++
 .../spark/sql/hive/CarbonHiveMetadataUtil.scala |   56 +
 .../apache/spark/sql/hive/CarbonMetastore.scala |  803 +++++++++++++
 .../sql/optimizer/CarbonLateDecodeRule.scala    |  796 +++++++++++++
 .../org/apache/spark/util/CleanFiles.scala      |   46 +
 .../org/apache/spark/util/Compaction.scala      |   46 +
 .../apache/spark/util/DeleteSegmentByDate.scala |   47 +
 .../apache/spark/util/DeleteSegmentById.scala   |   52 +
 .../org/apache/spark/util/ShowSegments.scala    |   82 ++
 .../org/apache/spark/util/TableAPIUtil.scala    |   54 +
 .../org/apache/spark/util/TableLoader.scala     |   93 ++
 .../carbondata/CarbonDataSourceSuite.scala      |   70 ++
 pom.xml                                         |   30 +-
 processing/pom.xml                              |    7 -
 .../lcm/status/SegmentStatusManager.java        |    7 +-
 63 files changed, 7402 insertions(+), 190 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/conf/dataload.properties.template
----------------------------------------------------------------------
diff --git a/conf/dataload.properties.template b/conf/dataload.properties.template
new file mode 100644
index 0000000..59cad4a
--- /dev/null
+++ b/conf/dataload.properties.template
@@ -0,0 +1,73 @@
+#
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.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 store path
+# you should change to the code path of your local machine
+carbon.storelocation=/Users/wangfei/code/incubator-carbondata/examples/spark2/target/store
+
+#true: use kettle to load data
+#false: use new flow to load data
+use_kettle=true
+
+# you should change to the code path of your local machine
+carbon.kettle.home=/Users/wangfei/code/incubator-carbondata/processing/carbonplugins
+
+#csv delimiter character
+delimiter=,
+
+#csv quote character
+#quotechar=\"
+
+#csv file header
+#fileheader=
+
+#csv data escape character
+#escapechar=\\
+
+#csv comment character
+#commentchar=#
+
+#column dictionary list
+#columndict=
+
+#null value's serialization format  
+#serialization_null_format=\\N
+
+#bad records logger
+#bad_records_logger_enable=false
+
+#bad records action
+#bad_records_action=force
+
+#all dictionary folder path
+#all_dictionary_path=
+
+#complex column's level 1 delimiter
+#complex_delimiter_level_1=\\$
+
+#complex column's level 2 delimiter
+#complex_delimiter_level_2=\\:
+
+#timestamp type column's data format
+#dateformat=
+
+#csv data whether support multiline
+#multiline=false
+
+#max number of csv file columns
+#maxColumns=

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 4f6d223..c2f0b9a 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -33,13 +33,6 @@
     <dev.path>${basedir}/../dev</dev.path>
   </properties>
 
-  <repositories>
-    <repository>
-      <id>pentaho-releases</id>
-      <url>http://repository.pentaho.org/artifactory/repo/</url>
-    </repository>
-  </repositories>
-
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala b/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala
index e8c437d..0489020 100644
--- a/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala
+++ b/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala
@@ -20,7 +20,7 @@ package org.apache.carbondata.examples
 import org.apache.spark.sql.{CarbonContext, CarbonEnv, CarbonRelation}
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
-import org.apache.carbondata.core.carbon.{CarbonTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.carbon.CarbonTableIdentifier
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.carbon.path.CarbonStorePath
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -63,9 +63,8 @@ object GenerateDictionaryExample {
                       dictFolderPath: String) {
     val dataBaseName = carbonTableIdentifier.getDatabaseName
     val tableName = carbonTableIdentifier.getTableName
-    val carbonRelation = CarbonEnv.get.carbonMetastore.
-      lookupRelation1(Option(dataBaseName),
-        tableName) (cc).asInstanceOf[CarbonRelation]
+    val carbonRelation = CarbonEnv.get.carbonMetastore.lookupRelation1(Option(dataBaseName),
+        tableName)(cc).asInstanceOf[CarbonRelation]
     val carbonTable = carbonRelation.tableMeta.carbonTable
     val dimensions = carbonTable.getDimensionByTableName(tableName.toLowerCase())
       .toArray.map(_.asInstanceOf[CarbonDimension])

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/examples/spark2/src/main/resources/data.csv
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/resources/data.csv b/examples/spark2/src/main/resources/data.csv
new file mode 100644
index 0000000..5d3169e
--- /dev/null
+++ b/examples/spark2/src/main/resources/data.csv
@@ -0,0 +1,11 @@
+shortField,intField,bigintField,doubleField,stringField,timestampField
+1, 10, 100, 48.4, spark, 2015/4/23
+5, 17, 140, 43.4, spark, 2015/7/27
+1, 11, 100, 44.4, flink, 2015/5/23
+1, 10, 150, 43.4, spark, 2015/7/24
+1, 10, 100, 47.4, spark, 2015/7/23
+3, 14, 160, 43.4, hive, 2015/7/26
+2, 10, 100, 43.4, impala, 2015/7/23
+1, 10, 100, 43.4, spark, 2015/5/23
+4, 16, 130, 42.4, impala, 2015/7/23
+1, 10, 100, 43.4, spark, 2015/7/23

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala
new file mode 100644
index 0000000..9102c78
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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.examples
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.util.TableLoader
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
+object CarbonExample {
+
+  def main(args: Array[String]): Unit = {
+    // to run the example, plz change this path to your local machine path
+    val rootPath = "/Users/wangfei/code/incubator-carbondata"
+    val spark = SparkSession
+        .builder()
+        .master("local")
+        .appName("CarbonExample")
+        .enableHiveSupport()
+        .config(CarbonCommonConstants.STORE_LOCATION,
+          s"$rootPath/examples/spark2/target/store")
+        .getOrCreate()
+    spark.sparkContext.setLogLevel("WARN")
+
+    // Drop table
+    spark.sql("DROP TABLE IF EXISTS carbon_table")
+    spark.sql("DROP TABLE IF EXISTS csv_table")
+
+    // Create table
+    spark.sql(
+      s"""
+         | CREATE TABLE carbon_table(
+         |    shortField short,
+         |    intField int,
+         |    bigintField long,
+         |    doubleField double,
+         |    stringField string
+         | )
+         | USING org.apache.spark.sql.CarbonSource
+       """.stripMargin)
+
+    val prop = s"$rootPath/conf/dataload.properties.template"
+    val tableName = "carbon_table"
+    val path = s"$rootPath/examples/spark2/src/main/resources/data.csv"
+    TableLoader.main(Array[String](prop, tableName, path))
+
+//    spark.sql(
+//      s"""
+//         | CREATE TABLE csv_table
+//         | (ID int,
+//         | date timestamp,
+//         | country string,
+//         | name string,
+//         | phonetype string,
+//         | serialname string,
+//         | salary int)
+//       """.stripMargin)
+//
+//    spark.sql(
+//      s"""
+//         | LOAD DATA LOCAL INPATH '$csvPath'
+//         | INTO TABLE csv_table
+//       """.stripMargin)
+
+//    spark.sql(
+//      s"""
+//         | INSERT INTO TABLE carbon_table
+//         | SELECT * FROM csv_table
+//       """.stripMargin)
+
+    // Perform a query
+//    spark.sql("""
+//           SELECT country, count(salary) AS amount
+//           FROM carbon_table
+//           WHERE country IN ('china','france')
+//           GROUP BY country
+//           """).show()
+
+    spark.sql("""
+             SELECT *
+             FROM carbon_table
+              """).show
+
+//    spark.sql("""
+//           SELECT sum(intField), stringField
+//           FROM carbon_table
+//           GROUP BY stringField
+//           """).show
+
+    // Drop table
+    spark.sql("DROP TABLE IF EXISTS carbon_table")
+    spark.sql("DROP TABLE IF EXISTS csv_table")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop/pom.xml b/hadoop/pom.xml
index 53061fb..b04c4c6 100644
--- a/hadoop/pom.xml
+++ b/hadoop/pom.xml
@@ -33,13 +33,6 @@
     <dev.path>${basedir}/../dev</dev.path>
   </properties>
 
-  <repositories>
-    <repository>
-      <id>pentaho-releases</id>
-      <url>http://repository.pentaho.org/artifactory/repo/</url>
-    </repository>
-  </repositories>
-
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonProjection.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonProjection.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonProjection.java
index 95afd2f..8cd539f 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonProjection.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonProjection.java
@@ -18,13 +18,16 @@
  */
 package org.apache.carbondata.hadoop;
 
+import java.io.Serializable;
 import java.util.LinkedHashSet;
 import java.util.Set;
 
 /**
  * User can add required columns
  */
-public class CarbonProjection {
+public class CarbonProjection implements Serializable {
+
+  private static final long serialVersionUID = -4328676723039530713L;
 
   private Set<String> columns = new LinkedHashSet<>();
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
index fa8ba6e..5eda4d8 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
@@ -70,6 +70,8 @@ public abstract class AbstractDictionaryDecodedReadSupport<T> implements CarbonR
         } catch (CarbonUtilException e) {
           throw new RuntimeException(e);
         }
+      } else {
+        dataTypes[i] = carbonColumns[i].getDataType();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration-testcases/pom.xml
----------------------------------------------------------------------
diff --git a/integration-testcases/pom.xml b/integration-testcases/pom.xml
index 2fbe41b..737e628 100644
--- a/integration-testcases/pom.xml
+++ b/integration-testcases/pom.xml
@@ -32,14 +32,6 @@
   <properties>
     <dev.path>${basedir}/../dev</dev.path>
   </properties>
-
-  <repositories>
-    <repository>
-      <id>pentaho-releases</id>
-      <url>http://repository.pentaho.org/artifactory/repo/</url>
-    </repository>
-  </repositories>
-
   <dependencies>
     <dependency>
       <groupId>com.databricks</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common/pom.xml b/integration/spark-common/pom.xml
index b0ab3ef..d3f42b4 100644
--- a/integration/spark-common/pom.xml
+++ b/integration/spark-common/pom.xml
@@ -35,11 +35,6 @@
 
   <dependencies>
     <dependency>
-      <groupId>com.databricks</groupId>
-      <artifactId>spark-csv_${scala.binary.version}</artifactId>
-      <version>1.2.0</version>
-    </dependency>
-    <dependency>
       <groupId>org.apache.carbondata</groupId>
       <artifactId>carbondata-common</artifactId>
       <version>${project.version}</version>
@@ -84,6 +79,11 @@
       <artifactId>junit</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.databricks</groupId>
+      <artifactId>spark-csv_${scala.binary.version}</artifactId>
+      <version>1.2.0</version>
+    </dependency>
+    <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
       <version>2.2.1</version>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
index ca33fac..753f485 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/CarbonCompactionUtil.java
@@ -38,7 +38,6 @@ import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.CarbonUtilException;
 
-
 /**
  * Utility Class for the Compaction Flow.
  */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
index bb8fc5c..4b1958d 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
@@ -57,12 +57,19 @@ public class SparkRowReadSupportImpl extends AbstractDictionaryDecodedReadSuppor
             break;
           default:
         }
-      } else if (carbonColumns[i].hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+      }
+      else if (carbonColumns[i].hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         //convert the long to timestamp in case of direct dictionary column
         if (DataType.TIMESTAMP == carbonColumns[i].getDataType()) {
           data[i] = new Timestamp((long) data[i] / 1000);
         }
       }
+//      else if(dataTypes[i].equals(DataType.INT)) {
+//        data[i] = ((Long)(data[i])).intValue();
+//      }
+//        else if(dataTypes[i].equals(DataType.SHORT)) {
+//        data[i] = ((Double)(data[i])).shortValue();
+//      }
     }
     return new GenericRow(data);
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
index 1d8d6b2..319d85c 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
@@ -37,8 +37,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.load.{BlockDetails, LoadMetadataDetails}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory}
 import org.apache.carbondata.processing.constants.DataProcessorConstants
-import org.apache.carbondata.processing.csvreaderstep.JavaRddIterator
-import org.apache.carbondata.processing.csvreaderstep.RddInputUtils
+import org.apache.carbondata.processing.csvreaderstep.{JavaRddIterator, RddInputUtils}
 import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.carbondata.processing.graphgenerator.GraphGenerator
 import org.apache.carbondata.processing.model.CarbonLoadModel

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataLoadPartitionCoalescer.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataLoadPartitionCoalescer.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataLoadPartitionCoalescer.scala
index af349a8..77402b4 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataLoadPartitionCoalescer.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataLoadPartitionCoalescer.scala
@@ -17,8 +17,8 @@
 
 package org.apache.spark.rdd
 
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, LinkedHashSet}
 import scala.collection.mutable
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, LinkedHashSet}
 
 import org.apache.spark.Partition
 import org.apache.spark.scheduler.TaskLocation

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 1c9d774..6766a39 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -22,8 +22,7 @@ import java.util
 import scala.collection.JavaConverters._
 import scala.collection.mutable.Map
 
-import org.apache.spark.sql.execution.command.ColumnProperty
-import org.apache.spark.sql.execution.command.Field
+import org.apache.spark.sql.execution.command.{ColumnProperty, Field}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.lcm.status.SegmentStatusManager

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index e650bfe..e578488 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -35,11 +35,11 @@ import org.apache.spark.util.FileUtils
 import org.apache.carbondata.common.factory.CarbonCommonFactory
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.cache.dictionary.Dictionary
+import org.apache.carbondata.core.carbon.CarbonTableIdentifier
 import org.apache.carbondata.core.carbon.metadata.datatype.DataType
 import org.apache.carbondata.core.carbon.metadata.encoder.Encoding
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.carbon.path.CarbonStorePath
-import org.apache.carbondata.core.carbon.CarbonTableIdentifier
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory
 import org.apache.carbondata.core.reader.CarbonDictionaryReader

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonTableIdentifierImplicit.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonTableIdentifierImplicit.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonTableIdentifierImplicit.scala
index 79c0cc8..d607523 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonTableIdentifierImplicit.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonTableIdentifierImplicit.scala
@@ -17,8 +17,6 @@
 
 package org.apache.spark.sql.catalyst
 
-import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
-
 /**
  * Implicit functions for [TableIdentifier]
  */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
index e08660c..7368bff 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.spark.sql.hive
 
 import java.net.{InetAddress, InterfaceAddress, NetworkInterface}
@@ -143,19 +144,15 @@ object DistributionUtil {
     nodes.distinct
   }
 
-  /**
-   * Requesting the extra executors other than the existing ones.
-   *
-   * @param sc
-   * @param numExecutors
-   * @return
-   */
+  // Hack for spark2 integration
+  var numExistingExecutors: Int = _
+
   def ensureExecutors(sc: SparkContext, numExecutors: Int): Boolean = {
     sc.schedulerBackend match {
       case b: CoarseGrainedSchedulerBackend =>
-        val requiredExecutors = numExecutors - b.numExistingExecutors
-        LOGGER.info(s"number of executors is =$numExecutors existing executors are =" +
-            s"${ b.numExistingExecutors }")
+        val requiredExecutors = numExecutors - numExistingExecutors
+        LOGGER.info(s"number of executors is = $numExecutors existing executors are = " +
+                    s"$numExistingExecutors")
         if (requiredExecutors > 0) {
           b.requestExecutors(requiredExecutors)
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark/pom.xml b/integration/spark/pom.xml
index 6476576..9492296 100644
--- a/integration/spark/pom.xml
+++ b/integration/spark/pom.xml
@@ -33,13 +33,6 @@
     <dev.path>${basedir}/../../dev</dev.path>
   </properties>
 
-  <repositories>
-    <repository>
-      <id>pentaho-releases</id>
-      <url>http://repository.pentaho.org/artifactory/repo/</url>
-    </repository>
-  </repositories>
-
   <dependencies>
     <dependency>
       <groupId>com.databricks</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
index d898c4f..681c0c8 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
@@ -23,6 +23,7 @@ import scala.collection.JavaConverters._
 import scala.language.implicitConversions
 
 import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.SchemaRDD
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions.AttributeReference

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonEnv.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
index 6cfbd5f..c473253 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -17,11 +17,9 @@
 
 package org.apache.spark.sql
 
-import org.apache.spark.sql.hive.CarbonMetastore
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.sql.hive.{CarbonMetastore, DistributionUtil}
 
-/**
- * Carbon Environment for unified context
- */
 case class CarbonEnv(carbonMetastore: CarbonMetastore)
 
 object CarbonEnv {
@@ -35,6 +33,10 @@ object CarbonEnv {
       val cc = sqlContext.asInstanceOf[CarbonContext]
       val catalog = new CarbonMetastore(cc, cc.storePath, cc.hiveClientInterface, "")
       carbonEnv = CarbonEnv(catalog)
+      DistributionUtil.numExistingExecutors = sqlContext.sparkContext.schedulerBackend match {
+        case b: CoarseGrainedSchedulerBackend => b.numExistingExecutors
+        case _ => 0
+      }
       initialized = true
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala
index 3fe4f22..848e752 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonScan.scala
@@ -22,7 +22,6 @@ import java.util.ArrayList
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
-import org.apache.hadoop.conf.Configuration
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala
index 4320598..c199dba 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSparkUtil.scala
@@ -31,7 +31,8 @@ object CarbonSparkUtil {
 
   def createSparkMeta(carbonTable: CarbonTable): CarbonMetaData = {
     val dimensionsAttr = carbonTable.getDimensionByTableName(carbonTable.getFactTableName)
-        .asScala.map(x => x.getColName) // wf : may be problem
+        .asScala.map(x => x.getColName)
+    // wf : may be problem
     val measureAttr = carbonTable.getMeasureByTableName(carbonTable.getFactTableName)
         .asScala.map(x => x.getColName)
     val dictionary =

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index 33e7ae7..ee56067 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -35,6 +35,11 @@
 
   <dependencies>
     <dependency>
+      <groupId>com.databricks</groupId>
+      <artifactId>spark-csv_${scala.binary.version}</artifactId>
+      <version>${spark.csv.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.carbondata</groupId>
       <artifactId>carbondata-common</artifactId>
       <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFromMetadata.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFromMetadata.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFromMetadata.java
new file mode 100644
index 0000000..0926e1c
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFromMetadata.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.
+ */
+
+/**
+ * Project Name  : Carbon
+ * Module Name   : CARBON Data Processor
+ * Author    : R00903928
+ * Created Date  : 21-Sep-2015
+ * FileName   : DeleteLoadFromMetadata.java
+ * Description   : Kettle step to generate MD Key
+ * Class Version  : 1.0
+ */
+package org.apache.carbondata.spark.load;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+
+public final class DeleteLoadFromMetadata {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DeleteLoadFromMetadata.class.getName());
+
+  private DeleteLoadFromMetadata() {
+
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/java/org/apache/carbondata/spark/util/CarbonQueryUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/util/CarbonQueryUtil.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/util/CarbonQueryUtil.java
new file mode 100644
index 0000000..04ef665
--- /dev/null
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/util/CarbonQueryUtil.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.util;
+
+import java.io.IOException;
+import java.util.*;
+
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastorage.store.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastorage.store.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory.FileType;
+import org.apache.carbondata.core.load.LoadMetadataDetails;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.scan.model.CarbonQueryPlan;
+import org.apache.carbondata.spark.partition.api.Partition;
+import org.apache.carbondata.spark.partition.api.impl.DefaultLoadBalancer;
+import org.apache.carbondata.spark.partition.api.impl.PartitionMultiFileImpl;
+import org.apache.carbondata.spark.partition.api.impl.QueryPartitionHelper;
+import org.apache.carbondata.spark.splits.TableSplit;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * This utilty parses the Carbon query plan to actual query model object.
+ */
+public final class CarbonQueryUtil {
+
+  private CarbonQueryUtil() {
+
+  }
+
+
+  /**
+   * It creates the one split for each region server.
+   */
+  public static synchronized TableSplit[] getTableSplits(String databaseName, String tableName,
+      CarbonQueryPlan queryPlan) throws IOException {
+
+    //Just create splits depends on locations of region servers
+    List<Partition> allPartitions = null;
+    if (queryPlan == null) {
+      allPartitions =
+          QueryPartitionHelper.getInstance().getAllPartitions(databaseName, tableName);
+    } else {
+      allPartitions =
+          QueryPartitionHelper.getInstance().getPartitionsForQuery(queryPlan);
+    }
+    TableSplit[] splits = new TableSplit[allPartitions.size()];
+    for (int i = 0; i < splits.length; i++) {
+      splits[i] = new TableSplit();
+      List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      Partition partition = allPartitions.get(i);
+      String location =
+              QueryPartitionHelper.getInstance().getLocation(partition, databaseName, tableName);
+      locations.add(location);
+      splits[i].setPartition(partition);
+      splits[i].setLocations(locations);
+    }
+
+    return splits;
+  }
+
+  /**
+   * It creates the one split for each region server.
+   */
+  public static TableSplit[] getTableSplitsForDirectLoad(String sourcePath) throws Exception {
+
+    //Just create splits depends on locations of region servers
+    FileType fileType = FileFactory.getFileType(sourcePath);
+    DefaultLoadBalancer loadBalancer = null;
+    List<Partition> allPartitions = getAllFilesForDataLoad(sourcePath);
+    loadBalancer = new DefaultLoadBalancer(new ArrayList<String>(), allPartitions);
+    TableSplit[] tblSplits = new TableSplit[allPartitions.size()];
+    for (int i = 0; i < tblSplits.length; i++) {
+      tblSplits[i] = new TableSplit();
+      List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      Partition partition = allPartitions.get(i);
+      String location = loadBalancer.getNodeForPartitions(partition);
+      locations.add(location);
+      tblSplits[i].setPartition(partition);
+      tblSplits[i].setLocations(locations);
+    }
+    return tblSplits;
+  }
+
+  /**
+   * It creates the one split for each region server.
+   */
+  public static TableSplit[] getPartitionSplits(String sourcePath, String[] nodeList,
+      int partitionCount) throws Exception {
+
+    //Just create splits depends on locations of region servers
+    FileType fileType = FileFactory.getFileType(sourcePath);
+    DefaultLoadBalancer loadBalancer = null;
+    List<Partition> allPartitions = getAllPartitions(sourcePath, fileType, partitionCount);
+    loadBalancer = new DefaultLoadBalancer(Arrays.asList(nodeList), allPartitions);
+    TableSplit[] splits = new TableSplit[allPartitions.size()];
+    for (int i = 0; i < splits.length; i++) {
+      splits[i] = new TableSplit();
+      List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      Partition partition = allPartitions.get(i);
+      String location = loadBalancer.getNodeForPartitions(partition);
+      locations.add(location);
+      splits[i].setPartition(partition);
+      splits[i].setLocations(locations);
+    }
+    return splits;
+  }
+
+  public static void getAllFiles(String sourcePath, List<String> partitionsFiles, FileType fileType)
+      throws Exception {
+
+    if (!FileFactory.isFileExist(sourcePath, fileType, false)) {
+      throw new Exception("Source file doesn't exist at path: " + sourcePath);
+    }
+
+    CarbonFile file = FileFactory.getCarbonFile(sourcePath, fileType);
+    if (file.isDirectory()) {
+      CarbonFile[] fileNames = file.listFiles(new CarbonFileFilter() {
+        @Override public boolean accept(CarbonFile pathname) {
+          return true;
+        }
+      });
+      for (int i = 0; i < fileNames.length; i++) {
+        getAllFiles(fileNames[i].getPath(), partitionsFiles, fileType);
+      }
+    } else {
+      // add only csv files
+      if (file.getName().endsWith("csv")) {
+        partitionsFiles.add(file.getPath());
+      }
+    }
+  }
+
+  /**
+   * split sourcePath by comma
+   */
+  public static void splitFilePath(String sourcePath, List<String> partitionsFiles,
+      String separator) {
+    if (StringUtils.isNotEmpty(sourcePath)) {
+      String[] files = sourcePath.split(separator);
+      for (String file : files) {
+        partitionsFiles.add(file);
+      }
+    }
+  }
+
+  private static List<Partition> getAllFilesForDataLoad(String sourcePath) throws Exception {
+    List<String> files = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    splitFilePath(sourcePath, files, CarbonCommonConstants.COMMA);
+    List<Partition> partitionList =
+        new ArrayList<Partition>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    Map<Integer, List<String>> partitionFiles = new HashMap<Integer, List<String>>();
+
+    partitionFiles.put(0, new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN));
+    partitionList.add(new PartitionMultiFileImpl(0 + "", partitionFiles.get(0)));
+
+    for (int i = 0; i < files.size(); i++) {
+      partitionFiles.get(i % 1).add(files.get(i));
+    }
+    return partitionList;
+  }
+
+  private static List<Partition> getAllPartitions(String sourcePath, FileType fileType,
+      int partitionCount) throws Exception {
+    List<String> files = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    splitFilePath(sourcePath, files, CarbonCommonConstants.COMMA);
+    int[] numberOfFilesPerPartition = getNumberOfFilesPerPartition(files.size(), partitionCount);
+    int startIndex = 0;
+    int endIndex = 0;
+    List<Partition> partitionList =
+        new ArrayList<Partition>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    if (numberOfFilesPerPartition != null) {
+      for (int i = 0; i < numberOfFilesPerPartition.length; i++) {
+        List<String> partitionFiles =
+            new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+        endIndex += numberOfFilesPerPartition[i];
+        for (int j = startIndex; j < endIndex; j++) {
+          partitionFiles.add(files.get(j));
+        }
+        startIndex += numberOfFilesPerPartition[i];
+        partitionList.add(new PartitionMultiFileImpl(i + "", partitionFiles));
+      }
+    }
+    return partitionList;
+  }
+
+  private static int[] getNumberOfFilesPerPartition(int numberOfFiles, int partitionCount) {
+    int div = numberOfFiles / partitionCount;
+    int mod = numberOfFiles % partitionCount;
+    int[] numberOfNodeToScan = null;
+    if (div > 0) {
+      numberOfNodeToScan = new int[partitionCount];
+      Arrays.fill(numberOfNodeToScan, div);
+    } else if (mod > 0) {
+      numberOfNodeToScan = new int[mod];
+    }
+    for (int i = 0; i < mod; i++) {
+      numberOfNodeToScan[i] = numberOfNodeToScan[i] + 1;
+    }
+    return numberOfNodeToScan;
+  }
+
+  public static List<String> getListOfSlices(LoadMetadataDetails[] details) {
+    List<String> slices = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    if (null != details) {
+      for (LoadMetadataDetails oneLoad : details) {
+        if (!CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals(oneLoad.getLoadStatus())) {
+          String loadName = CarbonCommonConstants.LOAD_FOLDER + oneLoad.getLoadName();
+          slices.add(loadName);
+        }
+      }
+    }
+    return slices;
+  }
+
+  /**
+   * This method will clear the dictionary cache for a given map of columns and dictionary cache
+   * mapping
+   *
+   * @param columnToDictionaryMap
+   */
+  public static void clearColumnDictionaryCache(Map<String, Dictionary> columnToDictionaryMap) {
+    for (Map.Entry<String, Dictionary> entry : columnToDictionaryMap.entrySet()) {
+      CarbonUtil.clearDictionaryCache(entry.getValue());
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala
new file mode 100644
index 0000000..ea97bca
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.spark
+
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+ /**
+  * Carbon column validator
+  */
+class CarbonColumnValidator extends ColumnValidator {
+  def validateColumns(allColumns: Seq[ColumnSchema]) {
+    allColumns.foreach { columnSchema =>
+      val colWithSameId = allColumns.filter { x =>
+        x.getColumnUniqueId.equals(columnSchema.getColumnUniqueId)
+      }
+      if (colWithSameId.size > 1) {
+        throw new MalformedCarbonCommandException("Two column can not have same columnId")
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
new file mode 100644
index 0000000..2cd4eb7
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
@@ -0,0 +1,391 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.optimizer.{AttributeReferenceWrapper}
+import org.apache.spark.sql.sources
+import org.apache.spark.sql.types.StructType
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn
+import org.apache.carbondata.scan.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
+import org.apache.carbondata.scan.expression.conditional._
+import org.apache.carbondata.scan.expression.logical.{AndExpression, FalseExpression, OrExpression}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * All filter conversions are done here.
+ */
+object CarbonFilters {
+
+  /**
+   * Converts data sources filters to carbon filter predicates.
+   */
+  def createCarbonFilter(schema: StructType,
+      predicate: sources.Filter): Option[CarbonExpression] = {
+    val dataTypeOf = schema.map(f => f.name -> f.dataType).toMap
+
+    def createFilter(predicate: sources.Filter): Option[CarbonExpression] = {
+      predicate match {
+
+        case sources.EqualTo(name, value) =>
+          Some(new EqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.Not(sources.EqualTo(name, value)) =>
+          Some(new NotEqualsExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+
+        case sources.EqualNullSafe(name, value) =>
+          Some(new EqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.Not(sources.EqualNullSafe(name, value)) =>
+          Some(new NotEqualsExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+
+        case sources.GreaterThan(name, value) =>
+          Some(new GreaterThanExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.LessThan(name, value) =>
+          Some(new LessThanExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.GreaterThanOrEqual(name, value) =>
+          Some(new GreaterThanEqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+        case sources.LessThanOrEqual(name, value) =>
+          Some(new LessThanEqualToExpression(getCarbonExpression(name),
+            getCarbonLiteralExpression(name, value)))
+
+        case sources.In(name, values) =>
+          Some(new InExpression(getCarbonExpression(name),
+            new ListExpression(
+              convertToJavaList(values.map(f => getCarbonLiteralExpression(name, f)).toList))))
+        case sources.Not(sources.In(name, values)) =>
+          Some(new NotInExpression(getCarbonExpression(name),
+            new ListExpression(
+              convertToJavaList(values.map(f => getCarbonLiteralExpression(name, f)).toList))))
+
+        case sources.And(lhs, rhs) =>
+          (createFilter(lhs) ++ createFilter(rhs)).reduceOption(new AndExpression(_, _))
+
+        case sources.Or(lhs, rhs) =>
+          for {
+            lhsFilter <- createFilter(lhs)
+            rhsFilter <- createFilter(rhs)
+          } yield {
+            new OrExpression(lhsFilter, rhsFilter)
+          }
+
+        case _ => None
+      }
+    }
+
+    def getCarbonExpression(name: String) = {
+      new CarbonColumnExpression(name,
+        CarbonScalaUtil.convertSparkToCarbonDataType(dataTypeOf(name)))
+    }
+
+    def getCarbonLiteralExpression(name: String, value: Any): CarbonExpression = {
+      new CarbonLiteralExpression(value,
+        CarbonScalaUtil.convertSparkToCarbonDataType(dataTypeOf(name)))
+    }
+
+    createFilter(predicate)
+  }
+
+
+  // Check out which filters can be pushed down to carbon, remaining can be handled in spark layer.
+  // Mostly dimension filters are only pushed down since it is faster in carbon.
+  def selectFilters(filters: Seq[Expression],
+      attrList: java.util.HashSet[AttributeReferenceWrapper],
+      aliasMap: CarbonAliasDecoderRelation): Unit = {
+    def translate(expr: Expression, or: Boolean = false): Option[sources.Filter] = {
+      expr match {
+        case or@ Or(left, right) =>
+
+          val leftFilter = translate(left, or = true)
+          val rightFilter = translate(right, or = true)
+          if (leftFilter.isDefined && rightFilter.isDefined) {
+            Some( sources.Or(leftFilter.get, rightFilter.get))
+          } else {
+            or.collect {
+              case attr: AttributeReference =>
+                attrList.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+            }
+            None
+          }
+
+        case And(left, right) =>
+          (translate(left) ++ translate(right)).reduceOption(sources.And)
+
+        case EqualTo(a: Attribute, Literal(v, t)) =>
+          Some(sources.EqualTo(a.name, v))
+        case EqualTo(l@Literal(v, t), a: Attribute) =>
+          Some(sources.EqualTo(a.name, v))
+        case EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
+          Some(sources.EqualTo(a.name, v))
+        case EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(sources.EqualTo(a.name, v))
+
+        case Not(EqualTo(a: Attribute, Literal(v, t))) => new
+            Some(sources.Not(sources.EqualTo(a.name, v)))
+        case Not(EqualTo(Literal(v, t), a: Attribute)) => new
+            Some(sources.Not(sources.EqualTo(a.name, v)))
+        case Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) => new
+            Some(sources.Not(sources.EqualTo(a.name, v)))
+        case Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) => new
+            Some(sources.Not(sources.EqualTo(a.name, v)))
+        case IsNotNull(a: Attribute) => Some(sources.IsNotNull(a.name))
+        case IsNull(a: Attribute) => Some(sources.IsNull(a.name))
+        case Not(In(a: Attribute, list)) if !list.exists(!_.isInstanceOf[Literal]) =>
+          val hSet = list.map(e => e.eval(EmptyRow))
+          Some(sources.Not(sources.In(a.name, hSet.toArray)))
+        case In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) =>
+          val hSet = list.map(e => e.eval(EmptyRow))
+          Some(sources.In(a.name, hSet.toArray))
+        case Not(In(Cast(a: Attribute, _), list))
+          if !list.exists(!_.isInstanceOf[Literal]) =>
+          val hSet = list.map(e => e.eval(EmptyRow))
+          Some(sources.Not(sources.In(a.name, hSet.toArray)))
+        case In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
+          val hSet = list.map(e => e.eval(EmptyRow))
+          Some(sources.In(a.name, hSet.toArray))
+
+        case GreaterThan(a: Attribute, Literal(v, t)) =>
+          Some(sources.GreaterThan(a.name, v))
+        case GreaterThan(Literal(v, t), a: Attribute) =>
+          Some(sources.LessThan(a.name, v))
+        case GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
+          Some(sources.GreaterThan(a.name, v))
+        case GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(sources.LessThan(a.name, v))
+
+        case LessThan(a: Attribute, Literal(v, t)) =>
+          Some(sources.LessThan(a.name, v))
+        case LessThan(Literal(v, t), a: Attribute) =>
+          Some(sources.GreaterThan(a.name, v))
+        case LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
+          Some(sources.LessThan(a.name, v))
+        case LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(sources.GreaterThan(a.name, v))
+
+        case GreaterThanOrEqual(a: Attribute, Literal(v, t)) =>
+          Some(sources.GreaterThanOrEqual(a.name, v))
+        case GreaterThanOrEqual(Literal(v, t), a: Attribute) =>
+          Some(sources.LessThanOrEqual(a.name, v))
+        case GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+          Some(sources.GreaterThanOrEqual(a.name, v))
+        case GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(sources.LessThanOrEqual(a.name, v))
+
+        case LessThanOrEqual(a: Attribute, Literal(v, t)) =>
+          Some(sources.LessThanOrEqual(a.name, v))
+        case LessThanOrEqual(Literal(v, t), a: Attribute) =>
+          Some(sources.GreaterThanOrEqual(a.name, v))
+        case LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+          Some(sources.LessThanOrEqual(a.name, v))
+        case LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(sources.GreaterThanOrEqual(a.name, v))
+
+        case others =>
+          if (!or) {
+            others.collect {
+              case attr: AttributeReference =>
+                attrList.add(AttributeReferenceWrapper(aliasMap.getOrElse(attr, attr)))
+            }
+          }
+          None
+      }
+    }
+    filters.flatMap(translate(_, false)).toArray
+  }
+
+  def processExpression(exprs: Seq[Expression],
+      attributesNeedToDecode: java.util.HashSet[AttributeReference],
+      unprocessedExprs: ArrayBuffer[Expression],
+      carbonTable: CarbonTable): Option[CarbonExpression] = {
+    def transformExpression(expr: Expression, or: Boolean = false): Option[CarbonExpression] = {
+      expr match {
+        case or@ Or(left, right) =>
+          val leftFilter = transformExpression(left, true)
+          val rightFilter = transformExpression(right, true)
+          if (leftFilter.isDefined && rightFilter.isDefined) {
+            Some(new OrExpression(leftFilter.get, rightFilter.get))
+          } else {
+            or.collect {
+              case attr: AttributeReference => attributesNeedToDecode.add(attr)
+            }
+            unprocessedExprs += or
+            None
+          }
+
+        case And(left, right) =>
+          (transformExpression(left) ++ transformExpression(right)).reduceOption(new
+              AndExpression(_, _))
+
+        case EqualTo(a: Attribute, l@Literal(v, t)) => new
+            Some(new EqualToExpression(transformExpression(a).get, transformExpression(l).get))
+        case EqualTo(l@Literal(v, t), a: Attribute) => new
+            Some(new EqualToExpression(transformExpression(a).get, transformExpression(l).get))
+        case EqualTo(Cast(a: Attribute, _), l@Literal(v, t)) => new
+            Some(new EqualToExpression(transformExpression(a).get, transformExpression(l).get))
+        case EqualTo(l@Literal(v, t), Cast(a: Attribute, _)) => new
+            Some(new EqualToExpression(transformExpression(a).get, transformExpression(l).get))
+
+        case Not(EqualTo(a: Attribute, l@Literal(v, t))) => new
+            Some(new NotEqualsExpression(transformExpression(a).get, transformExpression(l).get))
+        case Not(EqualTo(l@Literal(v, t), a: Attribute)) => new
+            Some(new NotEqualsExpression(transformExpression(a).get, transformExpression(l).get))
+        case Not(EqualTo(Cast(a: Attribute, _), l@Literal(v, t))) => new
+            Some(new NotEqualsExpression(transformExpression(a).get, transformExpression(l).get))
+        case Not(EqualTo(l@Literal(v, t), Cast(a: Attribute, _))) => new
+            Some(new NotEqualsExpression(transformExpression(a).get, transformExpression(l).get))
+        case IsNotNull(child: Attribute) =>
+            Some(new NotEqualsExpression(transformExpression(child).get,
+             transformExpression(Literal(null)).get, true))
+        case IsNull(child: Attribute) =>
+            Some(new EqualToExpression(transformExpression(child).get,
+             transformExpression(Literal(null)).get, true))
+        case Not(In(a: Attribute, list))
+          if !list.exists(!_.isInstanceOf[Literal]) =>
+          if (list.exists(x => isNullLiteral(x.asInstanceOf[Literal]))) {
+            Some(new FalseExpression(transformExpression(a).get))
+          } else {
+            Some(new NotInExpression(transformExpression(a).get,
+              new ListExpression(convertToJavaList(list.map(transformExpression(_).get)))))
+          }
+        case In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) =>
+          Some(new InExpression(transformExpression(a).get,
+            new ListExpression(convertToJavaList(list.map(transformExpression(_).get)))))
+        case Not(In(Cast(a: Attribute, _), list))
+          if !list.exists(!_.isInstanceOf[Literal]) =>
+          /* if any illogical expression comes in NOT IN Filter like
+           NOT IN('scala',NULL) this will be treated as false expression and will
+           always return no result. */
+          if (list.exists(x => isNullLiteral(x.asInstanceOf[Literal]))) {
+            Some(new FalseExpression(transformExpression(a).get))
+          } else {
+            Some(new NotInExpression(transformExpression(a).get, new ListExpression(
+              convertToJavaList(list.map(transformExpression(_).get)))))
+          }
+        case In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
+          Some(new InExpression(transformExpression(a).get,
+            new ListExpression(convertToJavaList(list.map(transformExpression(_).get)))))
+
+        case GreaterThan(a: Attribute, l@Literal(v, t)) =>
+          Some(new GreaterThanExpression(transformExpression(a).get, transformExpression(l).get))
+        case GreaterThan(Cast(a: Attribute, _), l@Literal(v, t)) =>
+          Some(new GreaterThanExpression(transformExpression(a).get, transformExpression(l).get))
+        case GreaterThan(l@Literal(v, t), a: Attribute) =>
+          Some(new LessThanExpression(transformExpression(a).get, transformExpression(l).get))
+        case GreaterThan(l@Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(new LessThanExpression(transformExpression(a).get, transformExpression(l).get))
+
+        case LessThan(a: Attribute, l@Literal(v, t)) =>
+          Some(new LessThanExpression(transformExpression(a).get, transformExpression(l).get))
+        case LessThan(Cast(a: Attribute, _), l@Literal(v, t)) =>
+          Some(new LessThanExpression(transformExpression(a).get, transformExpression(l).get))
+        case LessThan(l@Literal(v, t), a: Attribute) =>
+          Some(new GreaterThanExpression(transformExpression(a).get, transformExpression(l).get))
+        case LessThan(l@Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(new GreaterThanExpression(transformExpression(a).get, transformExpression(l).get))
+
+        case GreaterThanOrEqual(a: Attribute, l@Literal(v, t)) =>
+          Some(new GreaterThanEqualToExpression(transformExpression(a).get,
+            transformExpression(l).get))
+        case GreaterThanOrEqual(Cast(a: Attribute, _), l@Literal(v, t)) =>
+          Some(new GreaterThanEqualToExpression(transformExpression(a).get,
+            transformExpression(l).get))
+        case GreaterThanOrEqual(l@Literal(v, t), a: Attribute) =>
+          Some(new LessThanEqualToExpression(transformExpression(a).get,
+            transformExpression(l).get))
+        case GreaterThanOrEqual(l@Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(new LessThanEqualToExpression(transformExpression(a).get,
+            transformExpression(l).get))
+
+        case LessThanOrEqual(a: Attribute, l@Literal(v, t)) =>
+          Some(new LessThanEqualToExpression(transformExpression(a).get,
+            transformExpression(l).get))
+        case LessThanOrEqual(Cast(a: Attribute, _), l@Literal(v, t)) =>
+          Some(new LessThanEqualToExpression(transformExpression(a).get,
+            transformExpression(l).get))
+        case LessThanOrEqual(l@Literal(v, t), a: Attribute) =>
+          Some(new GreaterThanEqualToExpression(transformExpression(a).get,
+            transformExpression(l).get))
+        case LessThanOrEqual(l@Literal(v, t), Cast(a: Attribute, _)) =>
+          Some(new GreaterThanEqualToExpression(transformExpression(a).get,
+            transformExpression(l).get))
+
+        case AttributeReference(name, dataType, _, _) =>
+          Some(new CarbonColumnExpression(name,
+            CarbonScalaUtil.convertSparkToCarbonDataType(
+              getActualCarbonDataType(name, carbonTable))))
+        case Literal(name, dataType) => Some(new
+            CarbonLiteralExpression(name, CarbonScalaUtil.convertSparkToCarbonDataType(dataType)))
+        case Cast(left, right) if !left.isInstanceOf[Literal] => transformExpression(left)
+        case others =>
+          if (!or) {
+            others.collect {
+              case attr: AttributeReference => attributesNeedToDecode.add(attr)
+            }
+            unprocessedExprs += others
+          }
+          None
+      }
+    }
+    exprs.flatMap(transformExpression(_, false)).reduceOption(new AndExpression(_, _))
+  }
+  private def isNullLiteral(exp: Expression): Boolean = {
+    if (null != exp
+        &&  exp.isInstanceOf[Literal]
+        && (exp.asInstanceOf[Literal].dataType == org.apache.spark.sql.types.DataTypes.NullType)
+        || (exp.asInstanceOf[Literal].value == null)) {
+      true
+    } else {
+      false
+    }
+  }
+  private def getActualCarbonDataType(column: String, carbonTable: CarbonTable) = {
+    var carbonColumn: CarbonColumn =
+      carbonTable.getDimensionByName(carbonTable.getFactTableName, column)
+    val dataType = if (carbonColumn != null) {
+      carbonColumn.getDataType
+    } else {
+      carbonColumn = carbonTable.getMeasureByName(carbonTable.getFactTableName, column)
+      carbonColumn.getDataType match {
+        case DataType.INT => DataType.LONG
+        case DataType.LONG => DataType.LONG
+        case DataType.DECIMAL => DataType.DECIMAL
+        case _ => DataType.DOUBLE
+      }
+    }
+    CarbonScalaUtil.convertCarbonToSparkDataType(dataType)
+  }
+
+  // Convert scala list to java list, Cannot use scalaList.asJava as while deserializing it is
+  // not able find the classes inside scala list and gives ClassNotFoundException.
+  private def convertToJavaList(
+      scalaList: Seq[CarbonExpression]): java.util.List[CarbonExpression] = {
+    val javaList = new java.util.ArrayList[CarbonExpression]()
+    scalaList.foreach(javaList.add)
+    javaList
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
new file mode 100644
index 0000000..5f0c7e3
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
+/**
+ * Contains all options for Spark data source
+ */
+class CarbonOption(options: Map[String, String]) {
+  def tableIdentifier: String = options.getOrElse("tableName", s"$dbName.$tableName")
+
+  def dbName: String = options.getOrElse("dbName", CarbonCommonConstants.DATABASE_DEFAULT_NAME)
+
+  def tableName: String = options.getOrElse("tableName", "default_table")
+
+  def tableId: String = options.getOrElse("tableId", "default_table_id")
+
+  def tablePath: String = s"$dbName/$tableName"
+
+  def partitionCount: String = options.getOrElse("partitionCount", "1")
+
+  def partitionClass: String = {
+    options.getOrElse("partitionClass",
+      "org.apache.carbondata.spark.partition.api.impl.SampleDataPartitionerImpl")
+  }
+
+  def tempCSV: Boolean = options.getOrElse("tempCSV", "true").toBoolean
+
+  def compress: Boolean = options.getOrElse("compress", "false").toBoolean
+
+  def useKettle: Boolean = options.getOrElse("useKettle", "true").toBoolean
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
new file mode 100644
index 0000000..7618558
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import org.apache.carbondata.core.carbon.{CarbonTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.{CarbonDimension, ColumnSchema}
+
+
+ /**
+  * Column validator
+  */
+trait ColumnValidator {
+  def validateColumns(columns: Seq[ColumnSchema])
+}
+/**
+ * Dictionary related helper service
+ */
+trait DictionaryDetailService {
+  def getDictionaryDetail(dictFolderPath: String, primDimensions: Array[CarbonDimension],
+      table: CarbonTableIdentifier, storePath: String): DictionaryDetail
+}
+
+/**
+ * Dictionary related detail
+ */
+case class DictionaryDetail(columnIdentifiers: Array[ColumnIdentifier],
+    dictFilePaths: Array[String], dictFileExists: Array[Boolean])
+
+/**
+ * Factory class
+ */
+object CarbonSparkFactory {
+   /**
+    * @return column validator
+    */
+  def getCarbonColumnValidator(): ColumnValidator = {
+    new CarbonColumnValidator
+  }
+
+  /**
+   * @return dictionary helper
+   */
+  def getDictionaryDetailService(): DictionaryDetailService = {
+    new DictionaryDetailHelper
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
new file mode 100644
index 0000000..52457b8
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.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
+
+import scala.collection.mutable.HashMap
+
+import org.apache.carbondata.core.carbon.{CarbonTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.carbon.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.datastorage.store.filesystem.{CarbonFile, CarbonFileFilter}
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory
+
+class DictionaryDetailHelper extends DictionaryDetailService {
+  def getDictionaryDetail(dictfolderPath: String, primDimensions: Array[CarbonDimension],
+      table: CarbonTableIdentifier, storePath: String): DictionaryDetail = {
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, table)
+    val dictFilePaths = new Array[String](primDimensions.length)
+    val dictFileExists = new Array[Boolean](primDimensions.length)
+    val columnIdentifier = new Array[ColumnIdentifier](primDimensions.length)
+
+    val fileType = FileFactory.getFileType(dictfolderPath)
+    // Metadata folder
+    val metadataDirectory = FileFactory.getCarbonFile(dictfolderPath, fileType)
+    // need list all dictionary file paths with exists flag
+    val carbonFiles = metadataDirectory.listFiles(new CarbonFileFilter {
+      @Override def accept(pathname: CarbonFile): Boolean = {
+        CarbonTablePath.isDictionaryFile(pathname)
+      }
+    })
+    // 2 put dictionary file names to fileNamesMap
+    val fileNamesMap = new HashMap[String, Int]
+    for (i <- 0 until carbonFiles.length) {
+      fileNamesMap.put(carbonFiles(i).getName, i)
+    }
+    // 3 lookup fileNamesMap, if file name is in fileNamesMap, file is exists, or not.
+    primDimensions.zipWithIndex.foreach { f =>
+      columnIdentifier(f._2) = f._1.getColumnIdentifier
+      dictFilePaths(f._2) = carbonTablePath.getDictionaryFilePath(f._1.getColumnId)
+      dictFileExists(f._2) =
+        fileNamesMap.get(CarbonTablePath.getDictionaryFileName(f._1.getColumnId)) match {
+          case None => false
+          case Some(_) => true
+        }
+    }
+
+    DictionaryDetail(columnIdentifier, dictFilePaths, dictFileExists)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/590ecceb/integration/spark2/src/main/scala/org/apache/carbondata/spark/KeyVal.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/KeyVal.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/KeyVal.scala
new file mode 100644
index 0000000..254052b
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/KeyVal.scala
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+/**
+ * It is just Key value class. I don't get any other alternate to make the RDD class to
+ * work with my minimum knowledge in scala.
+ * May be I will remove later once I gain good knowledge :)
+ *
+ */
+
+package org.apache.carbondata.spark
+
+import org.apache.carbondata.core.load.LoadMetadataDetails
+
+trait Value[V] extends Serializable {
+  def getValue(value: Array[Object]): V
+}
+
+class ValueImpl extends Value[Array[Object]] {
+  override def getValue(value: Array[Object]): Array[Object] = value
+}
+
+trait RawValue[V] extends Serializable {
+  def getValue(value: Array[Any]): V
+}
+
+class RawValueImpl extends RawValue[Array[Any]] {
+  override def getValue(value: Array[Any]): Array[Any] = value
+}
+
+trait DataLoadResult[K, V] extends Serializable {
+  def getKey(key: String, value: LoadMetadataDetails): (K, V)
+}
+
+class DataLoadResultImpl extends DataLoadResult[String, LoadMetadataDetails] {
+  override def getKey(key: String, value: LoadMetadataDetails): (String, LoadMetadataDetails) = {
+    (key, value)
+  }
+}
+
+
+trait PartitionResult[K, V] extends Serializable {
+  def getKey(key: Int, value: Boolean): (K, V)
+
+}
+
+class PartitionResultImpl extends PartitionResult[Int, Boolean] {
+  override def getKey(key: Int, value: Boolean): (Int, Boolean) = (key, value)
+}
+
+trait MergeResult[K, V] extends Serializable {
+  def getKey(key: Int, value: Boolean): (K, V)
+
+}
+
+class MergeResultImpl extends MergeResult[Int, Boolean] {
+  override def getKey(key: Int, value: Boolean): (Int, Boolean) = (key, value)
+}
+
+trait DeletedLoadResult[K, V] extends Serializable {
+  def getKey(key: String, value: String): (K, V)
+}
+
+class DeletedLoadResultImpl extends DeletedLoadResult[String, String] {
+  override def getKey(key: String, value: String): (String, String) = (key, value)
+}
+
+trait RestructureResult[K, V] extends Serializable {
+  def getKey(key: Int, value: Boolean): (K, V)
+}
+
+class RestructureResultImpl extends RestructureResult[Int, Boolean] {
+  override def getKey(key: Int, value: Boolean): (Int, Boolean) = (key, value)
+}


[6/6] incubator-carbondata git commit: [CARBONDATA-473] integrate with spark 2 stable datasource api This closes #372

Posted by ja...@apache.org.
[CARBONDATA-473] integrate with spark 2 stable datasource api This closes #372


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

Branch: refs/heads/master
Commit: 286eccb5ec925925da7fbd394a51423edcda14eb
Parents: d94b99f 590ecce
Author: jackylk <ja...@huawei.com>
Authored: Thu Dec 1 01:46:15 2016 +0800
Committer: jackylk <ja...@huawei.com>
Committed: Thu Dec 1 01:46:15 2016 +0800

----------------------------------------------------------------------
 conf/dataload.properties.template               |   73 ++
 core/pom.xml                                    |    7 -
 .../examples/GenerateDictionaryExample.scala    |    7 +-
 examples/spark2/src/main/resources/data.csv     |   11 +
 .../carbondata/examples/CarbonExample.scala     |  109 ++
 hadoop/pom.xml                                  |    7 -
 .../carbondata/hadoop/CarbonProjection.java     |    5 +-
 .../AbstractDictionaryDecodedReadSupport.java   |    2 +
 integration-testcases/pom.xml                   |    8 -
 integration/spark-common/pom.xml                |   10 +-
 .../spark/merger/CarbonCompactionUtil.java      |    1 -
 .../readsupport/SparkRowReadSupportImpl.java    |    9 +-
 .../spark/rdd/CarbonDataLoadRDD.scala           |    3 +-
 .../spark/rdd/DataLoadPartitionCoalescer.scala  |    2 +-
 .../carbondata/spark/util/CommonUtil.scala      |    3 +-
 .../spark/util/GlobalDictionaryUtil.scala       |    2 +-
 .../CarbonTableIdentifierImplicit.scala         |    2 -
 .../spark/sql/hive/DistributionUtil.scala       |   17 +-
 integration/spark/pom.xml                       |    7 -
 .../spark/sql/CarbonDatasourceRelation.scala    |    1 +
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   10 +-
 .../scala/org/apache/spark/sql/CarbonScan.scala |    1 -
 .../org/apache/spark/sql/CarbonSparkUtil.scala  |    3 +-
 integration/spark2/pom.xml                      |    5 +
 .../spark/load/DeleteLoadFromMetadata.java      |   44 +
 .../carbondata/spark/util/CarbonQueryUtil.java  |  248 ++++
 .../spark/CarbonColumnValidator.scala           |   36 +
 .../apache/carbondata/spark/CarbonFilters.scala |  391 ++++++
 .../apache/carbondata/spark/CarbonOption.scala  |   48 +
 .../carbondata/spark/CarbonSparkFactory.scala   |   60 +
 .../spark/DictionaryDetailHelper.scala          |   62 +
 .../org/apache/carbondata/spark/KeyVal.scala    |   89 ++
 .../spark/rdd/CarbonDataRDDFactory.scala        | 1115 ++++++++++++++++++
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  250 ++++
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  305 +++++
 .../carbondata/spark/util/CarbonSparkUtil.scala |   45 +
 .../carbondata/spark/util/QueryPlanUtil.scala   |   56 +
 .../apache/spark/repl/CarbonSparkILoop.scala    |   72 ++
 .../spark/sql/CarbonCatalystOperators.scala     |   98 ++
 .../spark/sql/CarbonDataFrameWriter.scala       |  168 +++
 .../sql/CarbonDatasourceHadoopRelation.scala    |   78 ++
 .../spark/sql/CarbonDictionaryDecoder.scala     |  222 ++++
 .../scala/org/apache/spark/sql/CarbonEnv.scala  |   59 +
 .../scala/org/apache/spark/sql/CarbonScan.scala |   11 +-
 .../org/apache/spark/sql/CarbonSource.scala     |  143 +++
 .../spark/sql/SparkUnknownExpression.scala      |  130 ++
 .../org/apache/spark/sql/TableCreator.scala     |  490 ++++++++
 .../execution/CarbonLateDecodeStrategy.scala    |  128 +-
 .../execution/command/carbonTableSchema.scala   |  750 ++++++++++++
 .../spark/sql/hive/CarbonHiveMetadataUtil.scala |   56 +
 .../apache/spark/sql/hive/CarbonMetastore.scala |  803 +++++++++++++
 .../sql/optimizer/CarbonLateDecodeRule.scala    |  796 +++++++++++++
 .../org/apache/spark/util/CleanFiles.scala      |   46 +
 .../org/apache/spark/util/Compaction.scala      |   46 +
 .../apache/spark/util/DeleteSegmentByDate.scala |   47 +
 .../apache/spark/util/DeleteSegmentById.scala   |   52 +
 .../org/apache/spark/util/ShowSegments.scala    |   82 ++
 .../org/apache/spark/util/TableAPIUtil.scala    |   54 +
 .../org/apache/spark/util/TableLoader.scala     |   93 ++
 .../carbondata/CarbonDataSourceSuite.scala      |   70 ++
 pom.xml                                         |   30 +-
 processing/pom.xml                              |    7 -
 .../lcm/status/SegmentStatusManager.java        |    7 +-
 63 files changed, 7402 insertions(+), 190 deletions(-)
----------------------------------------------------------------------