You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by qi...@apache.org on 2017/09/27 06:34:04 UTC

[1/6] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Repository: carbondata
Updated Branches:
  refs/heads/master 2ec69f617 -> 6627cac0c


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
index 24b2981..9f4a8ce 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
@@ -21,11 +21,14 @@ import scala.collection.mutable
 import scala.language.implicitConversions
 
 import org.apache.spark.sql.{DeleteRecords, ShowLoadsCommand, UpdateTable}
-import org.apache.spark.sql.catalyst.{CarbonDDLSqlParser, TableIdentifier}
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.CarbonDDLSqlParser
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit._
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.management.{AlterTableCompactionCommand, CleanFilesCommand, DeleteLoadByIdCommand, DeleteLoadByLoadDateCommand, LoadTableCommand}
+import org.apache.spark.sql.execution.command.partition.{AlterTableDropCarbonPartitionCommand, AlterTableSplitCarbonPartitionCommand}
+import org.apache.spark.sql.execution.command.schema.{AlterTableAddColumnCommand, AlterTableDataTypeChangeCommand, AlterTableDropColumnCommand}
 import org.apache.spark.sql.types.StructField
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -45,10 +48,10 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
       initLexical
       phrase(start)(new lexical.Scanner(input)) match {
         case Success(plan, _) => plan match {
-          case x: LoadTable =>
+          case x: LoadTableCommand =>
             x.inputSqlString = input
             x
-          case x: AlterTableCompaction =>
+          case x: AlterTableCompactionCommand =>
             x.alterTableModel.alterSql = input
             x
           case logicalPlan => logicalPlan
@@ -80,7 +83,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
       case dbName ~ table ~ addInfo =>
         val alterTableAddPartitionModel =
           AlterTableSplitPartitionModel(dbName, table, "0", addInfo)
-        AlterTableSplitPartitionCommand(alterTableAddPartitionModel)
+        AlterTableSplitCarbonPartitionCommand(alterTableAddPartitionModel)
     }
 
   protected lazy val alterSplitPartition: Parser[LogicalPlan] =
@@ -92,7 +95,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         if (partitionId == 0) {
           sys.error("Please use [Alter Table Add Partition] statement to split default partition!")
         }
-        AlterTableSplitPartitionCommand(alterTableSplitPartitionModel)
+        AlterTableSplitCarbonPartitionCommand(alterTableSplitPartitionModel)
     }
 
   protected lazy val alterDropPartition: Parser[LogicalPlan] =
@@ -105,7 +108,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         }
         val alterTableDropPartitionModel =
           AlterTableDropPartitionModel(dbName, table, partitionId, dropWithData)
-        AlterTableDropPartition(alterTableDropPartitionModel)
+        AlterTableDropCarbonPartitionCommand(alterTableDropPartitionModel)
     }
 
 
@@ -115,7 +118,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         val altertablemodel =
           AlterTableModel(convertDbNameToLowerCase(dbName), table, None, compactType,
           Some(System.currentTimeMillis()), null)
-        AlterTableCompaction(altertablemodel)
+        AlterTableCompactionCommand(altertablemodel)
     }
 
   protected lazy val deleteRecords: Parser[LogicalPlan] =
@@ -253,7 +256,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
           validateOptions(optionsList)
         }
         val optionsMap = optionsList.getOrElse(List.empty[(String, String)]).toMap
-        LoadTable(convertDbNameToLowerCase(databaseNameOp), tableName, filePath, Seq(), optionsMap,
+        LoadTableCommand(
+          convertDbNameToLowerCase(databaseNameOp),
+          tableName,
+          filePath,
+          Seq(),
+          optionsMap,
           isOverwrite.isDefined)
     }
 
@@ -262,7 +270,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     (WHERE ~> (SEGMENT ~ "." ~ ID) ~> IN ~> "(" ~> repsep(segmentId, ",")) <~ ")" ~
     opt(";") ^^ {
       case dbName ~ tableName ~ loadids =>
-        DeleteLoadsById(loadids, dbName, tableName.toLowerCase())
+        DeleteLoadByIdCommand(loadids, dbName, tableName.toLowerCase())
     }
 
   protected lazy val deleteLoadsByLoadDate: Parser[LogicalPlan] =
@@ -272,7 +280,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
       case database ~ table ~ condition =>
         condition match {
           case dateField ~ dateValue =>
-            DeleteLoadsByLoadDate(convertDbNameToLowerCase(database),
+            DeleteLoadByLoadDateCommand(convertDbNameToLowerCase(database),
               table.toLowerCase(),
               dateField,
               dateValue)
@@ -282,14 +290,15 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
   protected lazy val cleanFiles: Parser[LogicalPlan] =
     CLEAN ~> FILES ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident <~ opt(";") ^^ {
       case databaseName ~ tableName =>
-        CleanFiles(convertDbNameToLowerCase(databaseName), tableName.toLowerCase())
+        CleanFilesCommand(convertDbNameToLowerCase(databaseName), tableName.toLowerCase())
     }
 
   protected lazy val explainPlan: Parser[LogicalPlan] =
     (EXPLAIN ~> opt(EXTENDED)) ~ startCommand ^^ {
       case isExtended ~ logicalPlan =>
         logicalPlan match {
-          case plan: CreateTable => ExplainCommand(logicalPlan, extended = isExtended.isDefined)
+          case _: CarbonCreateTableCommand =>
+            ExplainCommand(logicalPlan, extended = isExtended.isDefined)
           case _ => ExplainCommand(OneRowRelation)
         }
     }
@@ -317,7 +326,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
             table.toLowerCase,
             columnName.toLowerCase,
             columnNameCopy.toLowerCase)
-        AlterTableDataTypeChange(alterTableChangeDataTypeModel)
+        AlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel)
     }
 
   protected lazy val alterTableAddColumns: Parser[LogicalPlan] =
@@ -386,7 +395,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
           tableModel.dimCols,
           tableModel.msrCols,
           tableModel.highcardinalitydims.getOrElse(Seq.empty))
-        AlterTableAddColumns(alterTableAddColumnsModel)
+        AlterTableAddColumnCommand(alterTableAddColumnsModel)
     }
 
   private def checkFieldDefaultValue(fieldName: String, defaultValueColumnName: String): Boolean = {
@@ -410,7 +419,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         val alterTableDropColumnModel = AlterTableDropColumnModel(convertDbNameToLowerCase(dbName),
           table.toLowerCase,
           values.map(_.toLowerCase))
-        AlterTableDropColumns(alterTableDropColumnModel)
+        AlterTableDropColumnCommand(alterTableDropColumnModel)
     }
 
   def getFields(schema: Seq[StructField]): Seq[Field] = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index b4389a6..52008f2 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.parser.ParserUtils._
 import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{CreateTableContext, TablePropertyListContext}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SparkSqlAstBuilder
-import org.apache.spark.sql.execution.command.{BucketFields, CreateTable, Field, PartitionerField, TableModel}
+import org.apache.spark.sql.execution.command.{BucketFields, CarbonCreateTableCommand, Field, PartitionerField, TableModel}
 import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
 
 import org.apache.carbondata.core.util.{CarbonSessionInfo, SessionParams, ThreadLocalSessionInfo}
@@ -146,7 +146,7 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
         tableProperties,
         bucketFields)
 
-      CreateTable(tableModel)
+      CarbonCreateTableCommand(tableModel)
     } else {
       super.visitCreateTable(ctx)
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
index d00cb84..91121ce 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
@@ -17,7 +17,8 @@
 package org.apache.spark.util
 
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
-import org.apache.spark.sql.execution.command.{AlterTableCompaction, AlterTableModel}
+import org.apache.spark.sql.execution.command.AlterTableModel
+import org.apache.spark.sql.execution.command.management.AlterTableCompactionCommand
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.processing.merger.CompactionType
@@ -33,7 +34,7 @@ object Compaction {
     TableAPIUtil.validateTableExists(spark, dbName, tableName)
     if (compactionType.equalsIgnoreCase(CarbonCommonConstants.MAJOR) ||
         compactionType.equalsIgnoreCase(CarbonCommonConstants.MINOR)) {
-      AlterTableCompaction(AlterTableModel(Some(dbName),
+      AlterTableCompactionCommand(AlterTableModel(Some(dbName),
         tableName,
         None,
         compactionType,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
index 72c7426..501402b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
@@ -24,7 +24,7 @@ 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.spark.sql.execution.command.management.LoadTableCommand
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
@@ -61,7 +61,7 @@ object TableLoader {
 
   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, false).run(spark)
+    LoadTableCommand(dbName, tableName, inputPaths, Nil, options, false).run(spark)
   }
 
   def main(args: Array[String]): Unit = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala
index db62eb5..d53065f 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala
@@ -19,7 +19,7 @@ package org.apache.spark.carbondata.vectorreader
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.execution.command.LoadTable
+import org.apache.spark.sql.execution.command.management.LoadTableCommand
 import org.apache.spark.sql.execution.{BatchedDataSourceScanExec, RowDataSourceScanExec}
 import org.scalatest.BeforeAndAfterAll
 


[3/6] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by qi...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
new file mode 100644
index 0000000..897895a
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
@@ -0,0 +1,520 @@
+/*
+ * 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.management
+
+import scala.collection.JavaConverters._
+
+import org.apache.commons.lang3.StringUtils
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.execution.command.{DataLoadTableFileMapping, DataProcessCommand, RunnableCommand, UpdateTableModel}
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.util.{CausedBy, FileUtils}
+
+import org.apache.carbondata.common.constants.LoggerAction
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.dictionary.server.DictionaryServer
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format
+import org.apache.carbondata.processing.constants.TableOptionConstant
+import org.apache.carbondata.processing.etl.DataLoadingException
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
+import org.apache.carbondata.processing.newflow.exception.NoRetryException
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.load.ValidateUtil
+import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DictionaryLoadModel}
+import org.apache.carbondata.spark.util.{CommonUtil, GlobalDictionaryUtil}
+
+case class LoadTableCommand(
+    databaseNameOp: Option[String],
+    tableName: String,
+    factPathFromUser: String,
+    dimFilesPath: Seq[DataLoadTableFileMapping],
+    options: scala.collection.immutable.Map[String, String],
+    isOverwriteTable: Boolean,
+    var inputSqlString: String = null,
+    dataFrame: Option[DataFrame] = None,
+    updateModel: Option[UpdateTableModel] = None)
+  extends RunnableCommand with DataProcessCommand {
+
+  private def getFinalOptions(carbonProperty: CarbonProperties):
+  scala.collection.mutable.Map[String, String] = {
+    val optionsFinal = scala.collection.mutable.Map[String, String]()
+    optionsFinal.put("delimiter", options.getOrElse("delimiter", ","))
+    optionsFinal.put("quotechar", options.getOrElse("quotechar", "\""))
+    optionsFinal.put("fileheader", options.getOrElse("fileheader", ""))
+    optionsFinal.put("escapechar", options.getOrElse("escapechar", "\\"))
+    optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
+    optionsFinal.put("columndict", options.getOrElse("columndict", null))
+    optionsFinal
+      .put("serialization_null_format", options.getOrElse("serialization_null_format", "\\N"))
+    optionsFinal.put("bad_records_logger_enable", options.getOrElse("bad_records_logger_enable",
+      carbonProperty
+        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE,
+          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT)))
+    val badRecordActionValue = carbonProperty
+      .getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+        CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT)
+    optionsFinal.put("bad_records_action", options.getOrElse("bad_records_action", carbonProperty
+      .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION,
+        badRecordActionValue)))
+    optionsFinal
+      .put("is_empty_data_bad_record", options.getOrElse("is_empty_data_bad_record", carbonProperty
+        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD,
+          CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT)))
+    optionsFinal.put("all_dictionary_path", options.getOrElse("all_dictionary_path", ""))
+    optionsFinal
+      .put("complex_delimiter_level_1", options.getOrElse("complex_delimiter_level_1", "\\$"))
+    optionsFinal
+      .put("complex_delimiter_level_2", options.getOrElse("complex_delimiter_level_2", "\\:"))
+    optionsFinal.put("dateformat", options.getOrElse("dateformat",
+      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT,
+        CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)))
+
+    optionsFinal.put("global_sort_partitions", options.getOrElse("global_sort_partitions",
+      carbonProperty
+        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS, null)))
+
+    optionsFinal.put("maxcolumns", options.getOrElse("maxcolumns", null))
+
+    optionsFinal.put("batch_sort_size_inmb", options.getOrElse("batch_sort_size_inmb",
+      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB,
+        carbonProperty.getProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
+          CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))))
+
+    optionsFinal.put("bad_record_path", options.getOrElse("bad_record_path",
+      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
+        carbonProperty.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+          CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))))
+
+    val useOnePass = options.getOrElse("single_pass",
+      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
+        CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT)).trim.toLowerCase match {
+      case "true" =>
+        true
+      case "false" =>
+        // when single_pass = false  and if either alldictionarypath
+        // or columnDict is configured the do not allow load
+        if (StringUtils.isNotEmpty(optionsFinal("all_dictionary_path")) ||
+            StringUtils.isNotEmpty(optionsFinal("columndict"))) {
+          throw new MalformedCarbonCommandException(
+            "Can not use all_dictionary_path or columndict without single_pass.")
+        } else {
+          false
+        }
+      case illegal =>
+        val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+        LOGGER.error(s"Can't use single_pass, because illegal syntax found: [" + illegal + "] " +
+                     "Please set it as 'true' or 'false'")
+        false
+    }
+    optionsFinal.put("single_pass", useOnePass.toString)
+    optionsFinal
+  }
+
+  private def checkDefaultValue(value: String, default: String) = {
+    if (StringUtils.isEmpty(value)) {
+      default
+    } else {
+      value
+    }
+  }
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    if (dataFrame.isDefined && updateModel.isEmpty) {
+      val rdd = dataFrame.get.rdd
+      if (rdd.partitions == null || rdd.partitions.length == 0) {
+        LOGGER.warn("DataLoading finished. No data was loaded.")
+        return Seq.empty
+      }
+    }
+
+    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(Option(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+    if (null == relation.tableMeta.carbonTable) {
+      LOGGER.error(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 carbonProperty: CarbonProperties = CarbonProperties.getInstance()
+    carbonProperty.addProperty("zookeeper.enable.lock", "false")
+    val optionsFinal = getFinalOptions(carbonProperty)
+
+    val tableProperties = relation.tableMeta.carbonTable.getTableInfo
+      .getFactTable.getTableProperties
+
+    optionsFinal.put("sort_scope", tableProperties.getOrDefault("sort_scope",
+      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
+        carbonProperty.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
+          CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))))
+
+    try {
+      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.carbonTable.getStorePath)
+
+      val table = relation.tableMeta.carbonTable
+      carbonLoadModel.setTableName(table.getFactTableName)
+      val dataLoadSchema = new CarbonDataLoadSchema(table)
+      // Need to fill dimension relation
+      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+
+      val 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 sort_scope = optionsFinal("sort_scope")
+      val single_pass = optionsFinal("single_pass")
+      val bad_records_logger_enable = optionsFinal("bad_records_logger_enable")
+      val bad_records_action = optionsFinal("bad_records_action")
+      val bad_record_path = optionsFinal("bad_record_path")
+      val global_sort_partitions = optionsFinal("global_sort_partitions")
+      val dateFormat = optionsFinal("dateformat")
+      val delimeter = optionsFinal("delimiter")
+      val complex_delimeter_level1 = optionsFinal("complex_delimiter_level_1")
+      val complex_delimeter_level2 = optionsFinal("complex_delimiter_level_2")
+      val all_dictionary_path = optionsFinal("all_dictionary_path")
+      val column_dict = optionsFinal("columndict")
+      ValidateUtil.validateDateFormat(dateFormat, table, tableName)
+      ValidateUtil.validateSortScope(table, sort_scope)
+
+      if (bad_records_logger_enable.toBoolean ||
+          LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
+        if (!CarbonUtil.isValidBadStorePath(bad_record_path)) {
+          sys.error("Invalid bad records location.")
+        }
+      }
+      carbonLoadModel.setBadRecordsLocation(bad_record_path)
+
+      ValidateUtil.validateGlobalSortPartitions(global_sort_partitions)
+      carbonLoadModel.setEscapeChar(checkDefaultValue(optionsFinal("escapechar"), "\\"))
+      carbonLoadModel.setQuoteChar(checkDefaultValue(optionsFinal("quotechar"), "\""))
+      carbonLoadModel.setCommentChar(checkDefaultValue(optionsFinal("commentchar"), "#"))
+
+      // if there isn't file header in csv file and load sql doesn't provide FILEHEADER option,
+      // we should use table schema to generate file header.
+      var fileHeader = optionsFinal("fileheader")
+      val headerOption = options.get("header")
+      if (headerOption.isDefined) {
+        // whether the csv file has file header
+        // the default value is true
+        val header = try {
+          headerOption.get.toBoolean
+        } catch {
+          case ex: IllegalArgumentException =>
+            throw new MalformedCarbonCommandException(
+              "'header' option should be either 'true' or 'false'. " + ex.getMessage)
+        }
+        if (header) {
+          if (fileHeader.nonEmpty) {
+            throw new MalformedCarbonCommandException(
+              "When 'header' option is true, 'fileheader' option is not required.")
+          }
+        } else {
+          if (fileHeader.isEmpty) {
+            fileHeader = table.getCreateOrderColumn(table.getFactTableName)
+              .asScala.map(_.getColName).mkString(",")
+          }
+        }
+      }
+
+      carbonLoadModel.setDateFormat(dateFormat)
+      carbonLoadModel.setDefaultTimestampFormat(carbonProperty.getProperty(
+        CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+      carbonLoadModel.setDefaultDateFormat(carbonProperty.getProperty(
+        CarbonCommonConstants.CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+      carbonLoadModel
+        .setSerializationNullFormat(
+          TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + "," +
+          optionsFinal("serialization_null_format"))
+      carbonLoadModel
+        .setBadRecordsLoggerEnable(
+          TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName + "," + bad_records_logger_enable)
+      carbonLoadModel
+        .setBadRecordsAction(
+          TableOptionConstant.BAD_RECORDS_ACTION.getName + "," + bad_records_action)
+      carbonLoadModel
+        .setIsEmptyDataBadRecord(
+          DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," +
+          optionsFinal("is_empty_data_bad_record"))
+      carbonLoadModel.setSortScope(sort_scope)
+      carbonLoadModel.setBatchSortSizeInMb(optionsFinal("batch_sort_size_inmb"))
+      carbonLoadModel.setGlobalSortPartitions(global_sort_partitions)
+      carbonLoadModel.setUseOnePass(single_pass.toBoolean)
+      if (delimeter.equalsIgnoreCase(complex_delimeter_level1) ||
+          complex_delimeter_level1.equalsIgnoreCase(complex_delimeter_level2) ||
+          delimeter.equalsIgnoreCase(complex_delimeter_level2)) {
+        sys.error(s"Field Delimiter & Complex types delimiter are same")
+      } else {
+        carbonLoadModel.setComplexDelimiterLevel1(
+          CarbonUtil.delimiterConverter(complex_delimeter_level1))
+        carbonLoadModel.setComplexDelimiterLevel2(
+          CarbonUtil.delimiterConverter(complex_delimeter_level2))
+      }
+      // set local dictionary path, and dictionary file extension
+      carbonLoadModel.setAllDictPath(all_dictionary_path)
+
+      val 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(delimeter))
+        carbonLoadModel.setCsvHeader(fileHeader)
+        carbonLoadModel.setColDictFilePath(column_dict)
+        carbonLoadModel.setDirectLoad(true)
+        carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+        val validatedMaxColumns = CommonUtil.validateMaxColumns(carbonLoadModel.getCsvHeaderColumns,
+          optionsFinal("maxcolumns"))
+        carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
+        GlobalDictionaryUtil.updateTableMetadataFunc = updateTableMetadata
+        val storePath = relation.tableMeta.storePath
+        // add the start entry for the new load in the table status file
+        if (updateModel.isEmpty) {
+          CommonUtil.
+            readAndUpdateLoadProgressInTableMeta(carbonLoadModel, storePath, isOverwriteTable)
+        }
+        if (isOverwriteTable) {
+          LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
+        }
+        if (null == carbonLoadModel.getLoadMetadataDetails) {
+          CommonUtil.readLoadMetadataDetails(carbonLoadModel)
+        }
+        if (carbonLoadModel.getLoadMetadataDetails.isEmpty && carbonLoadModel.getUseOnePass &&
+            StringUtils.isEmpty(column_dict) && StringUtils.isEmpty(all_dictionary_path)) {
+          LOGGER.info(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
+          LOGGER.audit(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
+          carbonLoadModel.setUseOnePass(false)
+        }
+        // Create table and metadata folders if not exist
+        val carbonTablePath = CarbonStorePath
+          .getCarbonTablePath(storePath, table.getCarbonTableIdentifier)
+        val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+        val fileType = FileFactory.getFileType(metadataDirectoryPath)
+        if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
+          FileFactory.mkdirs(metadataDirectoryPath, fileType)
+        }
+        if (carbonLoadModel.getUseOnePass) {
+          val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+          val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+            .getCarbonTableIdentifier
+          val carbonTablePath = CarbonStorePath
+            .getCarbonTablePath(storePath, carbonTableIdentifier)
+          val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
+          val dimensions = carbonTable.getDimensionByTableName(
+            carbonTable.getFactTableName).asScala.toArray
+          val colDictFilePath = carbonLoadModel.getColDictFilePath
+          if (!StringUtils.isEmpty(colDictFilePath)) {
+            carbonLoadModel.initPredefDictMap()
+            // generate predefined dictionary
+            GlobalDictionaryUtil
+              .generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
+                dimensions, carbonLoadModel, sparkSession.sqlContext, storePath, dictFolderPath)
+          }
+          if (!StringUtils.isEmpty(all_dictionary_path)) {
+            carbonLoadModel.initPredefDictMap()
+            GlobalDictionaryUtil
+              .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
+                carbonLoadModel,
+                storePath,
+                carbonTableIdentifier,
+                dictFolderPath,
+                dimensions,
+                all_dictionary_path)
+          }
+          // dictionaryServerClient dictionary generator
+          val dictionaryServerPort = carbonProperty
+            .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
+              CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
+          val sparkDriverHost = sparkSession.sqlContext.sparkContext.
+            getConf.get("spark.driver.host")
+          carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
+          // start dictionary server when use one pass load and dimension with DICTIONARY
+          // encoding is present.
+          val allDimensions = table.getAllDimensions.asScala.toList
+          val createDictionary = allDimensions.exists {
+            carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
+                               !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
+          }
+          val server: Option[DictionaryServer] = if (createDictionary) {
+            val dictionaryServer = DictionaryServer
+              .getInstance(dictionaryServerPort.toInt, carbonTable)
+            carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
+            sparkSession.sparkContext.addSparkListener(new SparkListener() {
+              override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
+                dictionaryServer.shutdown()
+              }
+            })
+            Some(dictionaryServer)
+          } else {
+            None
+          }
+          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+            carbonLoadModel,
+            relation.tableMeta.storePath,
+            columnar,
+            partitionStatus,
+            server,
+            isOverwriteTable,
+            dataFrame,
+            updateModel)
+        } else {
+          val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
+            val fields = dataFrame.get.schema.fields
+            import org.apache.spark.sql.functions.udf
+            // extracting only segment from tupleId
+            val getSegIdUDF = udf((tupleId: String) =>
+              CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID))
+            // getting all fields except tupleId field as it is not required in the value
+            var otherFields = fields.toSeq
+              .filter(field => !field.name
+                .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
+              .map(field => new Column(field.name))
+
+            // extract tupleId field which will be used as a key
+            val segIdColumn = getSegIdUDF(new Column(UnresolvedAttribute
+              .quotedString(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))).
+              as(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID)
+            // use dataFrameWithoutTupleId as dictionaryDataFrame
+            val dataFrameWithoutTupleId = dataFrame.get.select(otherFields: _*)
+            otherFields = otherFields :+ segIdColumn
+            // use dataFrameWithTupleId as loadDataFrame
+            val dataFrameWithTupleId = dataFrame.get.select(otherFields: _*)
+            (Some(dataFrameWithoutTupleId), Some(dataFrameWithTupleId))
+          } else {
+            (dataFrame, dataFrame)
+          }
+
+          GlobalDictionaryUtil.generateGlobalDictionary(
+            sparkSession.sqlContext,
+            carbonLoadModel,
+            relation.tableMeta.storePath,
+            dictionaryDataFrame)
+          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+            carbonLoadModel,
+            relation.tableMeta.storePath,
+            columnar,
+            partitionStatus,
+            None,
+            isOverwriteTable,
+            loadDataFrame,
+            updateModel)
+        }
+      } catch {
+        case CausedBy(ex: NoRetryException) =>
+          LOGGER.error(ex, s"Dataload failure for $dbName.$tableName")
+          throw new RuntimeException(s"Dataload failure for $dbName.$tableName, ${ex.getMessage}")
+        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
+    }
+    Seq.empty
+  }
+
+  private def updateTableMetadata(
+      carbonLoadModel: CarbonLoadModel,
+      sqlContext: SQLContext,
+      model: DictionaryLoadModel,
+      noDictDimension: Array[CarbonDimension]): Unit = {
+    val sparkSession = sqlContext.sparkSession
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(model.hdfsLocation,
+      model.table)
+
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    // read TableInfo
+    val tableInfo: format.TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+
+    // modify TableInfo
+    val columns = tableInfo.getFact_table.getTable_columns
+    for (i <- 0 until columns.size) {
+      if (noDictDimension.exists(x => columns.get(i).getColumn_id.equals(x.getColumnId))) {
+        columns.get(i).encoders.remove(org.apache.carbondata.format.Encoding.DICTIONARY)
+      }
+    }
+    val entry = tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
+    entry.setTime_stamp(System.currentTimeMillis())
+
+    // write TableInfo
+    metastore.updateTableSchema(carbonTablePath.getCarbonTableIdentifier,
+      carbonTablePath.getCarbonTableIdentifier,
+      tableInfo, entry, carbonTablePath.getPath)(sparkSession)
+
+    // update the schema modified time
+    metastore.updateAndTouchSchemasUpdatedTime(model.hdfsLocation)
+
+    // update CarbonDataLoadSchema
+    val carbonTable = metastore.lookupRelation(Option(model.table.getDatabaseName),
+      model.table.getTableName)(sqlContext.sparkSession).asInstanceOf[CarbonRelation].tableMeta
+      .carbonTable
+    carbonLoadModel.setCarbonDataLoadSchema(new CarbonDataLoadSchema(carbonTable))
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
new file mode 100644
index 0000000..02c7023
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
@@ -0,0 +1,322 @@
+/*
+ * 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.mutation
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command.ExecutionErrors
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, DeleteDeltaBlockDetails, SegmentUpdateDetails, TupleIdEnum}
+import org.apache.carbondata.core.mutate.data.RowCountDetailsVO
+import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager
+import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
+import org.apache.carbondata.hadoop.CarbonInputFormat
+import org.apache.carbondata.processing.exception.MultipleMatchingException
+import org.apache.carbondata.spark.DeleteDelataResultImpl
+import org.apache.carbondata.spark.load.FailureCauses
+import org.apache.carbondata.spark.util.QueryPlanUtil
+
+object DeleteExecution {
+  val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def getTableIdentifier(tableIdentifier: Seq[String]): TableIdentifier = {
+    if (tableIdentifier.size > 1) {
+      TableIdentifier(tableIdentifier(1), Some(tableIdentifier(0)))
+    } else {
+      TableIdentifier(tableIdentifier(0), None)
+    }
+  }
+
+  def deleteDeltaExecution(
+      identifier: Seq[String],
+      sparkSession: SparkSession,
+      dataRdd: RDD[Row],
+      timestamp: String, relation: CarbonRelation, isUpdateOperation: Boolean,
+      executorErrors: ExecutionErrors
+  ): Boolean = {
+
+    var res: Array[List[(String, (SegmentUpdateDetails, ExecutionErrors))]] = null
+    val tableName = getTableIdentifier(identifier).table
+    val database = GetDB.getDatabaseName(getTableIdentifier(identifier).database, sparkSession)
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(DeleteExecution.getTableIdentifier(identifier))(sparkSession).
+      asInstanceOf[CarbonRelation]
+
+    val storeLocation = relation.tableMeta.storePath
+    val absoluteTableIdentifier: AbsoluteTableIdentifier = new
+        AbsoluteTableIdentifier(storeLocation,
+          relation.tableMeta.carbonTableIdentifier)
+    val tablePath = CarbonStorePath.getCarbonTablePath(
+      storeLocation,
+      absoluteTableIdentifier.getCarbonTableIdentifier)
+    val factPath = tablePath.getFactDir
+
+    val carbonTable = relation.tableMeta.carbonTable
+    var deleteStatus = true
+    val deleteRdd = if (isUpdateOperation) {
+      val schema =
+        org.apache.spark.sql.types.StructType(Seq(org.apache.spark.sql.types.StructField(
+          CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID,
+          org.apache.spark.sql.types.StringType)))
+      val rdd = dataRdd
+        .map(row => Row(row.get(row.fieldIndex(
+          CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))))
+      sparkSession.createDataFrame(rdd, schema).rdd
+      // sqlContext.createDataFrame(rdd, schema).rdd
+    } else {
+      dataRdd
+    }
+
+    val (carbonInputFormat, job) =
+      QueryPlanUtil.createCarbonInputFormat(absoluteTableIdentifier)
+    CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
+    val keyRdd = deleteRdd.map({ row =>
+      val tupleId: String = row
+        .getString(row.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
+      val key = CarbonUpdateUtil.getSegmentWithBlockFromTID(tupleId)
+      (key, row)
+    }).groupByKey()
+
+    // if no loads are present then no need to do anything.
+    if (keyRdd.partitions.length == 0) {
+      return true
+    }
+
+    val blockMappingVO = carbonInputFormat.getBlockRowCount(job, absoluteTableIdentifier)
+    val segmentUpdateStatusMngr = new SegmentUpdateStatusManager(absoluteTableIdentifier)
+    CarbonUpdateUtil
+      .createBlockDetailsMap(blockMappingVO, segmentUpdateStatusMngr)
+
+    val rowContRdd =
+      sparkSession.sparkContext.parallelize(
+        blockMappingVO.getCompleteBlockRowDetailVO.asScala.toSeq,
+        keyRdd.partitions.length)
+
+    val rdd = rowContRdd.join(keyRdd)
+    res = rdd.mapPartitionsWithIndex(
+      (index: Int, records: Iterator[((String), (RowCountDetailsVO, Iterable[Row]))]) =>
+        Iterator[List[(String, (SegmentUpdateDetails, ExecutionErrors))]] {
+
+          var result = List[(String, (SegmentUpdateDetails, ExecutionErrors))]()
+          while (records.hasNext) {
+            val ((key), (rowCountDetailsVO, groupedRows)) = records.next
+            result = result ++
+                     deleteDeltaFunc(index,
+                       key,
+                       groupedRows.toIterator,
+                       timestamp,
+                       rowCountDetailsVO)
+          }
+          result
+        }
+    ).collect()
+
+    // if no loads are present then no need to do anything.
+    if (res.isEmpty) {
+      return true
+    }
+
+    // update new status file
+    checkAndUpdateStatusFiles()
+
+    // all or none : update status file, only if complete delete opeartion is successfull.
+    def checkAndUpdateStatusFiles(): Unit = {
+      val blockUpdateDetailsList = new util.ArrayList[SegmentUpdateDetails]()
+      val segmentDetails = new util.HashSet[String]()
+      res.foreach(resultOfSeg => resultOfSeg.foreach(
+        resultOfBlock => {
+          if (resultOfBlock._1.equalsIgnoreCase(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)) {
+            blockUpdateDetailsList.add(resultOfBlock._2._1)
+            segmentDetails.add(resultOfBlock._2._1.getSegmentName)
+            // if this block is invalid then decrement block count in map.
+            if (CarbonUpdateUtil.isBlockInvalid(resultOfBlock._2._1.getStatus)) {
+              CarbonUpdateUtil.decrementDeletedBlockCount(resultOfBlock._2._1,
+                blockMappingVO.getSegmentNumberOfBlockMapping)
+            }
+          }
+          else {
+            deleteStatus = false
+            // In case of failure , clean all related delete delta files
+            CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
+            LOGGER.audit(s"Delete data operation is failed for ${ database }.${ tableName }")
+            val errorMsg =
+              "Delete data operation is failed due to failure in creating delete delta file for " +
+              "segment : " + resultOfBlock._2._1.getSegmentName + " block : " +
+              resultOfBlock._2._1.getBlockName
+            executorErrors.failureCauses = resultOfBlock._2._2.failureCauses
+            executorErrors.errorMsg = resultOfBlock._2._2.errorMsg
+
+            if (executorErrors.failureCauses == FailureCauses.NONE) {
+              executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
+              executorErrors.errorMsg = errorMsg
+            }
+            LOGGER.error(errorMsg)
+            return
+          }
+        }
+      )
+      )
+
+      val listOfSegmentToBeMarkedDeleted = CarbonUpdateUtil
+        .getListOfSegmentsToMarkDeleted(blockMappingVO.getSegmentNumberOfBlockMapping)
+
+
+
+      // this is delete flow so no need of putting timestamp in the status file.
+      if (CarbonUpdateUtil
+            .updateSegmentStatus(blockUpdateDetailsList, carbonTable, timestamp, false) &&
+          CarbonUpdateUtil
+            .updateTableMetadataStatus(segmentDetails,
+              carbonTable,
+              timestamp,
+              !isUpdateOperation,
+              listOfSegmentToBeMarkedDeleted)
+      ) {
+        LOGGER.info(s"Delete data operation is successful for ${ database }.${ tableName }")
+        LOGGER.audit(s"Delete data operation is successful for ${ database }.${ tableName }")
+      }
+      else {
+        // In case of failure , clean all related delete delta files
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
+
+        val errorMessage = "Delete data operation is failed due to failure " +
+                           "in table status updation."
+        LOGGER.audit(s"Delete data operation is failed for ${ database }.${ tableName }")
+        LOGGER.error("Delete data operation is failed due to failure in table status updation.")
+        executorErrors.failureCauses = FailureCauses.STATUS_FILE_UPDATION_FAILURE
+        executorErrors.errorMsg = errorMessage
+        // throw new Exception(errorMessage)
+      }
+    }
+
+    def deleteDeltaFunc(index: Int,
+        key: String,
+        iter: Iterator[Row],
+        timestamp: String,
+        rowCountDetailsVO: RowCountDetailsVO
+    ): Iterator[(String, (SegmentUpdateDetails, ExecutionErrors))] = {
+
+      val result = new DeleteDelataResultImpl()
+      var deleteStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
+      val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+      // here key = segment/blockName
+      val blockName = CarbonUpdateUtil
+        .getBlockName(
+          CarbonTablePath.addDataPartPrefix(key.split(CarbonCommonConstants.FILE_SEPARATOR)(1)))
+      val segmentId = key.split(CarbonCommonConstants.FILE_SEPARATOR)(0)
+      val deleteDeltaBlockDetails: DeleteDeltaBlockDetails = new DeleteDeltaBlockDetails(blockName)
+      val resultIter = new Iterator[(String, (SegmentUpdateDetails, ExecutionErrors))] {
+        val segmentUpdateDetails = new SegmentUpdateDetails()
+        var TID = ""
+        var countOfRows = 0
+        try {
+          while (iter.hasNext) {
+            val oneRow = iter.next
+            TID = oneRow
+              .get(oneRow.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID)).toString
+            val offset = CarbonUpdateUtil.getRequiredFieldFromTID(TID, TupleIdEnum.OFFSET)
+            val blockletId = CarbonUpdateUtil
+              .getRequiredFieldFromTID(TID, TupleIdEnum.BLOCKLET_ID)
+            val pageId = Integer.parseInt(CarbonUpdateUtil
+              .getRequiredFieldFromTID(TID, TupleIdEnum.PAGE_ID))
+            val IsValidOffset = deleteDeltaBlockDetails.addBlocklet(blockletId, offset, pageId)
+            // stop delete operation
+            if(!IsValidOffset) {
+              executorErrors.failureCauses = FailureCauses.MULTIPLE_INPUT_ROWS_MATCHING
+              executorErrors.errorMsg = "Multiple input rows matched for same row."
+              throw new MultipleMatchingException("Multiple input rows matched for same row.")
+            }
+            countOfRows = countOfRows + 1
+          }
+
+          val blockPath = CarbonUpdateUtil.getTableBlockPath(TID, factPath)
+          val completeBlockName = CarbonTablePath
+            .addDataPartPrefix(CarbonUpdateUtil.getRequiredFieldFromTID(TID, TupleIdEnum.BLOCK_ID) +
+                               CarbonCommonConstants.FACT_FILE_EXT)
+          val deleteDeletaPath = CarbonUpdateUtil
+            .getDeleteDeltaFilePath(blockPath, blockName, timestamp)
+          val carbonDeleteWriter = new CarbonDeleteDeltaWriterImpl(deleteDeletaPath,
+            FileFactory.getFileType(deleteDeletaPath))
+
+
+
+          segmentUpdateDetails.setBlockName(blockName)
+          segmentUpdateDetails.setActualBlockName(completeBlockName)
+          segmentUpdateDetails.setSegmentName(segmentId)
+          segmentUpdateDetails.setDeleteDeltaEndTimestamp(timestamp)
+          segmentUpdateDetails.setDeleteDeltaStartTimestamp(timestamp)
+
+          val alreadyDeletedRows: Long = rowCountDetailsVO.getDeletedRowsInBlock
+          val totalDeletedRows: Long = alreadyDeletedRows + countOfRows
+          segmentUpdateDetails.setDeletedRowsInBlock(totalDeletedRows.toString)
+          if (totalDeletedRows == rowCountDetailsVO.getTotalNumberOfRows) {
+            segmentUpdateDetails.setStatus(CarbonCommonConstants.MARKED_FOR_DELETE)
+          }
+          else {
+            // write the delta file
+            carbonDeleteWriter.write(deleteDeltaBlockDetails)
+          }
+
+          deleteStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
+        } catch {
+          case e : MultipleMatchingException =>
+            LOGGER.audit(e.getMessage)
+            LOGGER.error(e.getMessage)
+          // dont throw exception here.
+          case e: Exception =>
+            val errorMsg = s"Delete data operation is failed for ${ database }.${ tableName }."
+            LOGGER.audit(errorMsg)
+            LOGGER.error(errorMsg + e.getMessage)
+            throw e
+        }
+
+
+        var finished = false
+
+        override def hasNext: Boolean = {
+          if (!finished) {
+            finished = true
+            finished
+          }
+          else {
+            !finished
+          }
+        }
+
+        override def next(): (String, (SegmentUpdateDetails, ExecutionErrors)) = {
+          finished = true
+          result.getKey(deleteStatus, (segmentUpdateDetails, executorErrors))
+        }
+      }
+      resultIter
+    }
+
+    true
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
new file mode 100644
index 0000000..34daf4e
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
@@ -0,0 +1,246 @@
+/*
+ * 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.mutation
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.execution.command.AlterTableModel
+import org.apache.spark.sql.execution.command.management.AlterTableCompactionCommand
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager
+import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CarbonDataMergerUtilResult, CompactionType}
+
+object HorizontalCompaction {
+
+  val LOG: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+
+  /**
+   * The method does horizontal compaction. After Update and Delete completion
+   * tryHorizontal compaction will be called. In case this method is called after
+   * Update statement then Update Compaction followed by Delete Compaction will be
+   * processed whereas for tryHorizontalCompaction called after Delete statement
+   * then only Delete Compaction will be processed.
+   */
+  def tryHorizontalCompaction(
+      sparkSession: SparkSession,
+      carbonRelation: CarbonRelation,
+      isUpdateOperation: Boolean): Unit = {
+
+    if (!CarbonDataMergerUtil.isHorizontalCompactionEnabled) {
+      return
+    }
+
+    var compactionTypeIUD = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
+    val carbonTable = carbonRelation.tableMeta.carbonTable
+    val absTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+    val updateTimeStamp = System.currentTimeMillis()
+    // To make sure that update and delete timestamps are not same,
+    // required to commit to status metadata and cleanup
+    val deleteTimeStamp = updateTimeStamp + 1
+
+    // get the valid segments
+    var segLists = CarbonDataMergerUtil.getValidSegmentList(absTableIdentifier)
+
+    if (segLists == null || segLists.size() == 0) {
+      return
+    }
+
+    // Should avoid reading Table Status file from Disk every time. Better to load it
+    // in-memory at the starting and pass it along the routines. The constructor of
+    // SegmentUpdateStatusManager reads the Table Status File and Table Update Status
+    // file and save the content in segmentDetails and updateDetails respectively.
+    val segmentUpdateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
+      absTableIdentifier)
+
+    if (isUpdateOperation) {
+
+      // This is only update operation, perform only update compaction.
+      compactionTypeIUD = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
+      performUpdateDeltaCompaction(sparkSession,
+        compactionTypeIUD,
+        carbonTable,
+        absTableIdentifier,
+        segmentUpdateStatusManager,
+        updateTimeStamp,
+        segLists)
+    }
+
+    // After Update Compaction perform delete compaction
+    compactionTypeIUD = CompactionType.IUD_DELETE_DELTA_COMPACTION
+    segLists = CarbonDataMergerUtil.getValidSegmentList(absTableIdentifier)
+    if (segLists == null || segLists.size() == 0) {
+      return
+    }
+
+    // Delete Compaction
+    performDeleteDeltaCompaction(sparkSession,
+      compactionTypeIUD,
+      carbonTable,
+      absTableIdentifier,
+      segmentUpdateStatusManager,
+      deleteTimeStamp,
+      segLists)
+  }
+
+  /**
+   * Update Delta Horizontal Compaction.
+   */
+  private def performUpdateDeltaCompaction(sparkSession: SparkSession,
+      compactionTypeIUD: CompactionType,
+      carbonTable: CarbonTable,
+      absTableIdentifier: AbsoluteTableIdentifier,
+      segmentUpdateStatusManager: SegmentUpdateStatusManager,
+      factTimeStamp: Long,
+      segLists: util.List[String]): Unit = {
+    val db = carbonTable.getDatabaseName
+    val table = carbonTable.getFactTableName
+    // get the valid segments qualified for update compaction.
+    val validSegList = CarbonDataMergerUtil.getSegListIUDCompactionQualified(segLists,
+      absTableIdentifier,
+      segmentUpdateStatusManager,
+      compactionTypeIUD)
+
+    if (validSegList.size() == 0) {
+      return
+    }
+
+    LOG.info(s"Horizontal Update Compaction operation started for [$db.$table].")
+    LOG.audit(s"Horizontal Update Compaction operation started for [$db.$table].")
+
+    try {
+      // Update Compaction.
+      val alterTableModel = AlterTableModel(Option(carbonTable.getDatabaseName),
+        carbonTable.getFactTableName,
+        Some(segmentUpdateStatusManager),
+        CompactionType.IUD_UPDDEL_DELTA_COMPACTION.toString,
+        Some(factTimeStamp),
+        "")
+
+      AlterTableCompactionCommand(alterTableModel).run(sparkSession)
+    }
+    catch {
+      case e: Exception =>
+        val msg = if (null != e.getMessage) {
+          e.getMessage
+        } else {
+          "Please check logs for more info"
+        }
+        throw new HorizontalCompactionException(
+          s"Horizontal Update Compaction Failed for [${ db }.${ table }]. " + msg, factTimeStamp)
+    }
+    LOG.info(s"Horizontal Update Compaction operation completed for [${ db }.${ table }].")
+    LOG.audit(s"Horizontal Update Compaction operation completed for [${ db }.${ table }].")
+  }
+
+  /**
+   * Delete Delta Horizontal Compaction.
+   */
+  private def performDeleteDeltaCompaction(sparkSession: SparkSession,
+      compactionTypeIUD: CompactionType,
+      carbonTable: CarbonTable,
+      absTableIdentifier: AbsoluteTableIdentifier,
+      segmentUpdateStatusManager: SegmentUpdateStatusManager,
+      factTimeStamp: Long,
+      segLists: util.List[String]): Unit = {
+
+    val db = carbonTable.getDatabaseName
+    val table = carbonTable.getFactTableName
+    val deletedBlocksList = CarbonDataMergerUtil.getSegListIUDCompactionQualified(segLists,
+      absTableIdentifier,
+      segmentUpdateStatusManager,
+      compactionTypeIUD)
+
+    if (deletedBlocksList.size() == 0) {
+      return
+    }
+
+    LOG.info(s"Horizontal Delete Compaction operation started for [$db.$table].")
+    LOG.audit(s"Horizontal Delete Compaction operation started for [$db.$table].")
+
+    try {
+
+      // Delete Compaction RDD
+      val rdd1 = sparkSession.sparkContext
+        .parallelize(deletedBlocksList.asScala, deletedBlocksList.size())
+
+      val timestamp = factTimeStamp
+      val updateStatusDetails = segmentUpdateStatusManager.getUpdateStatusDetails
+      val result = rdd1.mapPartitions(iter =>
+        new Iterator[Seq[CarbonDataMergerUtilResult]] {
+          override def hasNext: Boolean = iter.hasNext
+
+          override def next(): Seq[CarbonDataMergerUtilResult] = {
+            val segmentAndBlocks = iter.next
+            val segment = segmentAndBlocks.substring(0, segmentAndBlocks.lastIndexOf("/"))
+            val blockName = segmentAndBlocks
+              .substring(segmentAndBlocks.lastIndexOf("/") + 1, segmentAndBlocks.length)
+
+            val result = CarbonDataMergerUtil.compactBlockDeleteDeltaFiles(segment, blockName,
+              absTableIdentifier,
+              updateStatusDetails,
+              timestamp)
+
+            result.asScala.toList
+
+          }
+        }).collect
+
+      val resultList = ListBuffer[CarbonDataMergerUtilResult]()
+      result.foreach(x => {
+        x.foreach(y => {
+          resultList += y
+        })
+      })
+
+      val updateStatus = CarbonDataMergerUtil.updateStatusFile(resultList.toList.asJava,
+        carbonTable,
+        timestamp.toString,
+        segmentUpdateStatusManager)
+      if (updateStatus == false) {
+        LOG.audit(s"Delete Compaction data operation is failed for [$db.$table].")
+        LOG.error("Delete Compaction data operation is failed.")
+        throw new HorizontalCompactionException(
+          s"Horizontal Delete Compaction Failed for [$db.$table] ." +
+          s" Please check logs for more info.", factTimeStamp)
+      }
+      else {
+        LOG.info(s"Horizontal Delete Compaction operation completed for [$db.$table].")
+        LOG.audit(s"Horizontal Delete Compaction operation completed for [$db.$table].")
+      }
+    }
+    catch {
+      case e: Exception =>
+        val msg = if (null != e.getMessage) {
+          e.getMessage
+        } else {
+          "Please check logs for more info"
+        }
+        throw new HorizontalCompactionException(
+          s"Horizontal Delete Compaction Failed for [${ db }.${ table }]. " + msg, factTimeStamp)
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompactionException.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompactionException.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompactionException.scala
new file mode 100644
index 0000000..60ae078
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompactionException.scala
@@ -0,0 +1,24 @@
+/*
+ * 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.mutation
+
+class HorizontalCompactionException(
+    message: String,
+    // required for cleanup
+    val compactionTimeStamp: Long) extends RuntimeException(message) {
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
new file mode 100644
index 0000000..9cf8a91
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
@@ -0,0 +1,105 @@
+/*
+ * 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.mutation
+
+import org.apache.spark.sql.{CarbonEnv, Dataset, Row, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil
+import org.apache.carbondata.spark.load.FailureCauses
+
+/**
+ * IUD update delete and compaction framework.
+ *
+ */
+private[sql] case class ProjectForDeleteCommand(
+    plan: LogicalPlan,
+    identifier: Seq[String],
+    timestamp: String) extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+    val dataFrame = Dataset.ofRows(sparkSession, plan)
+    //    dataFrame.show(truncate = false)
+    //    dataFrame.collect().foreach(println)
+    val dataRdd = dataFrame.rdd
+
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(DeleteExecution.getTableIdentifier(identifier))(sparkSession).
+      asInstanceOf[CarbonRelation]
+    val carbonTable = relation.tableMeta.carbonTable
+    val metadataLock = CarbonLockFactory
+      .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+        LockUsage.METADATA_LOCK)
+    var lockStatus = false
+    try {
+      lockStatus = metadataLock.lockWithRetries()
+      LOGGER.audit(s" Delete data request has been received " +
+                   s"for ${ relation.databaseName }.${ relation.tableName }.")
+      if (lockStatus) {
+        LOGGER.info("Successfully able to get the table metadata file lock")
+      } else {
+        throw new Exception("Table is locked for deletion. Please try after some time")
+      }
+      val executorErrors = ExecutionErrors(FailureCauses.NONE, "")
+
+      // handle the clean up of IUD.
+      CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, false)
+
+      if (DeleteExecution
+        .deleteDeltaExecution(identifier, sparkSession, dataRdd, timestamp, relation,
+          isUpdateOperation = false, executorErrors)) {
+        // call IUD Compaction.
+        HorizontalCompaction.tryHorizontalCompaction(sparkSession, relation,
+          isUpdateOperation = false)
+      }
+    } catch {
+      case e: HorizontalCompactionException =>
+        LOGGER.error("Delete operation passed. Exception in Horizontal Compaction." +
+                     " Please check logs. " + e.getMessage)
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, e.compactionTimeStamp.toString)
+
+      case e: Exception =>
+        LOGGER.error(e, "Exception in Delete data operation " + e.getMessage)
+        // ****** start clean up.
+        // In case of failure , clean all related delete delta files
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
+
+        // clean up. Null check is required as for executor error some times message is null
+        if (null != e.getMessage) {
+          sys.error("Delete data operation is failed. " + e.getMessage)
+        }
+        else {
+          sys.error("Delete data operation is failed. Please check logs.")
+        }
+    } finally {
+      if (lockStatus) {
+        CarbonLockUtil.fileUnlock(metadataLock, LockUsage.METADATA_LOCK)
+      }
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
new file mode 100644
index 0000000..036ca49
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
@@ -0,0 +1,228 @@
+/*
+ * 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.mutation
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.management.LoadTableCommand
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.storage.StorageLevel
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.spark.load.FailureCauses
+
+private[sql] case class ProjectForUpdateCommand(
+    plan: LogicalPlan, tableIdentifier: Seq[String])
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER = LogServiceFactory.getLogService(ProjectForUpdateCommand.getClass.getName)
+
+    //  sqlContext.sparkContext.setLocalProperty(org.apache.spark.sql.execution.SQLExecution
+    //  .EXECUTION_ID_KEY, null)
+    // DataFrame(sqlContext, plan).show(truncate = false)
+    // return Seq.empty
+
+
+    val res = plan find {
+      case relation: LogicalRelation if relation.relation
+        .isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        true
+      case _ => false
+    }
+
+    if (res.isEmpty) {
+      return Seq.empty
+    }
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(DeleteExecution.getTableIdentifier(tableIdentifier))(sparkSession).
+      asInstanceOf[CarbonRelation]
+    //    val relation = CarbonEnv.get.carbonMetastore
+    //      .lookupRelation1(deleteExecution.getTableIdentifier(tableIdentifier))(sqlContext).
+    //      asInstanceOf[CarbonRelation]
+    val carbonTable = relation.tableMeta.carbonTable
+    val metadataLock = CarbonLockFactory
+      .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+        LockUsage.METADATA_LOCK)
+    var lockStatus = false
+    // get the current time stamp which should be same for delete and update.
+    val currentTime = CarbonUpdateUtil.readCurrentTime
+    //    var dataFrame: DataFrame = null
+    var dataSet: DataFrame = null
+    var isPersistEnabled = CarbonProperties.getInstance.isPersistUpdateDataset()
+    try {
+      lockStatus = metadataLock.lockWithRetries()
+      if (lockStatus) {
+        logInfo("Successfully able to get the table metadata file lock")
+      }
+      else {
+        throw new Exception("Table is locked for updation. Please try after some time")
+      }
+      val tablePath = CarbonStorePath.getCarbonTablePath(
+        carbonTable.getStorePath,
+        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier)
+      // Get RDD.
+
+      dataSet = if (isPersistEnabled) {
+        Dataset.ofRows(sparkSession, plan).persist(StorageLevel.fromString(
+          CarbonProperties.getInstance.getUpdateDatasetStorageLevel()))
+      }
+      else {
+        Dataset.ofRows(sparkSession, plan)
+      }
+      var executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
+
+
+      // handle the clean up of IUD.
+      CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, false)
+
+      // do delete operation.
+      DeleteExecution.deleteDeltaExecution(tableIdentifier, sparkSession, dataSet.rdd,
+        currentTime + "",
+        relation, isUpdateOperation = true, executionErrors)
+
+      if(executionErrors.failureCauses != FailureCauses.NONE) {
+        throw new Exception(executionErrors.errorMsg)
+      }
+
+      // do update operation.
+      performUpdate(dataSet, tableIdentifier, plan, sparkSession, currentTime, executionErrors)
+
+      if(executionErrors.failureCauses != FailureCauses.NONE) {
+        throw new Exception(executionErrors.errorMsg)
+      }
+
+      // Do IUD Compaction.
+      HorizontalCompaction.tryHorizontalCompaction(sparkSession, relation, isUpdateOperation = true)
+    } catch {
+      case e: HorizontalCompactionException =>
+        LOGGER.error(
+          "Update operation passed. Exception in Horizontal Compaction. Please check logs." + e)
+        // In case of failure , clean all related delta files
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, e.compactionTimeStamp.toString)
+
+      case e: Exception =>
+        LOGGER.error("Exception in update operation" + e)
+        // ****** start clean up.
+        // In case of failure , clean all related delete delta files
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, currentTime + "")
+
+        // *****end clean up.
+        if (null != e.getMessage) {
+          sys.error("Update operation failed. " + e.getMessage)
+        }
+        if (null != e.getCause && null != e.getCause.getMessage) {
+          sys.error("Update operation failed. " + e.getCause.getMessage)
+        }
+        sys.error("Update operation failed. please check logs.")
+    }
+    finally {
+      if (null != dataSet && isPersistEnabled) {
+        dataSet.unpersist()
+      }
+      if (lockStatus) {
+        CarbonLockUtil.fileUnlock(metadataLock, LockUsage.METADATA_LOCK)
+      }
+    }
+    Seq.empty
+  }
+
+  private def performUpdate(
+      dataFrame: Dataset[Row],
+      tableIdentifier: Seq[String],
+      plan: LogicalPlan,
+      sparkSession: SparkSession,
+      currentTime: Long,
+      executorErrors: ExecutionErrors): Unit = {
+
+    def isDestinationRelation(relation: CarbonDatasourceHadoopRelation): Boolean = {
+
+      val tableName = relation.identifier.getCarbonTableIdentifier.getTableName
+      val dbName = relation.identifier.getCarbonTableIdentifier.getDatabaseName
+      (tableIdentifier.size > 1 &&
+       tableIdentifier(0) == dbName &&
+       tableIdentifier(1) == tableName) ||
+      (tableIdentifier(0) == tableName)
+    }
+    def getHeader(relation: CarbonDatasourceHadoopRelation, plan: LogicalPlan): String = {
+      var header = ""
+      var found = false
+
+      plan match {
+        case Project(pList, _) if (!found) =>
+          found = true
+          header = pList
+            .filter(field => !field.name
+              .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
+            .map(col => if (col.name.endsWith(CarbonCommonConstants.UPDATED_COL_EXTENSION)) {
+              col.name
+                .substring(0, col.name.lastIndexOf(CarbonCommonConstants.UPDATED_COL_EXTENSION))
+            }
+            else {
+              col.name
+            }).mkString(",")
+      }
+      header
+    }
+    val ex = dataFrame.queryExecution.analyzed
+    val res = ex find {
+      case relation: LogicalRelation
+        if relation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+           isDestinationRelation(relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]) =>
+        true
+      case _ => false
+    }
+    val carbonRelation: CarbonDatasourceHadoopRelation = res match {
+      case Some(relation: LogicalRelation) =>
+        relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+      case _ => sys.error("")
+    }
+
+    val updateTableModel = UpdateTableModel(true, currentTime, executorErrors)
+
+    val header = getHeader(carbonRelation, plan)
+
+    LoadTableCommand(
+      Some(carbonRelation.identifier.getCarbonTableIdentifier.getDatabaseName),
+      carbonRelation.identifier.getCarbonTableIdentifier.getTableName,
+      null,
+      Seq(),
+      Map(("fileheader" -> header)),
+      false,
+      null,
+      Some(dataFrame),
+      Some(updateTableModel)).run(sparkSession)
+
+    executorErrors.errorMsg = updateTableModel.executorErrors.errorMsg
+    executorErrors.failureCauses = updateTableModel.executorErrors.failureCauses
+
+    Seq.empty
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/package.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/package.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/package.scala
new file mode 100644
index 0000000..07ef555
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/package.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command
+
+import scala.language.implicitConversions
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+object Checker {
+  def validateTableExists(
+      dbName: Option[String],
+      tableName: String,
+      session: SparkSession): Unit = {
+    val identifier = TableIdentifier(tableName, dbName)
+    if (!CarbonEnv.getInstance(session).carbonMetastore.tableExists(identifier)(session)) {
+      val err = s"table $dbName.$tableName not found"
+      LogServiceFactory.getLogService(this.getClass.getName).error(err)
+      throw new IllegalArgumentException(err)
+    }
+  }
+}
+
+/**
+ * Interface for command that modifies schema
+ */
+trait SchemaProcessCommand {
+  def processSchema(sparkSession: SparkSession): Seq[Row]
+}
+
+/**
+ * Interface for command that need to process data in file system
+ */
+trait DataProcessCommand {
+  def processData(sparkSession: SparkSession): Seq[Row]
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
new file mode 100644
index 0000000..1f06aed
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
@@ -0,0 +1,176 @@
+/*
+ * 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.partition
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableDropPartitionModel, DataProcessCommand, RunnableCommand, SchemaProcessCommand}
+import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.cache.CacheProvider
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.PartitionInfo
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+
+case class AlterTableDropCarbonPartitionCommand(
+    model: AlterTableDropPartitionModel)
+  extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
+  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    if (model.partitionId.equals("0")) {
+      sys.error(s"Cannot drop default partition! Please use delete statement!")
+    }
+    processSchema(sparkSession)
+    processData(sparkSession)
+    Seq.empty
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+    val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
+    val tableName = model.tableName
+    val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
+      .asInstanceOf[CarbonRelation]
+    val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+    val storePath = relation.tableMeta.storePath
+    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
+    if (relation == null) {
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
+      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
+      sys.error(s"Alter table failed. table not found: $dbName.$tableName")
+    }
+    val table = relation.tableMeta.carbonTable
+    val partitionInfo = table.getPartitionInfo(tableName)
+    if (partitionInfo == null) {
+      sys.error(s"Table $tableName is not a partition table.")
+    }
+    val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
+    // keep a copy of partitionIdList before update partitionInfo.
+    // will be used in partition data scan
+    oldPartitionIds.addAll(partitionIds.asJava)
+    val partitionIndex = partitionIds.indexOf(Integer.valueOf(model.partitionId))
+    partitionInfo.getPartitionType match {
+      case PartitionType.HASH => sys.error(s"Hash partition cannot be dropped!")
+      case PartitionType.RANGE =>
+        val rangeInfo = new util.ArrayList(partitionInfo.getRangeInfo)
+        val rangeToRemove = partitionInfo.getRangeInfo.get(partitionIndex - 1)
+        rangeInfo.remove(rangeToRemove)
+        partitionInfo.setRangeInfo(rangeInfo)
+      case PartitionType.LIST =>
+        val listInfo = new util.ArrayList(partitionInfo.getListInfo)
+        val listToRemove = partitionInfo.getListInfo.get(partitionIndex - 1)
+        listInfo.remove(listToRemove)
+        partitionInfo.setListInfo(listInfo)
+      case PartitionType.RANGE_INTERVAL =>
+        sys.error(s"Dropping range interval partition isn't support yet!")
+    }
+    partitionInfo.dropPartition(partitionIndex)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
+    val schemaFilePath = carbonTablePath.getSchemaFilePath
+    // read TableInfo
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
+
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
+    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
+      dbName, tableName, storePath)
+    val tableSchema = wrapperTableInfo.getFactTable
+    tableSchema.setPartitionInfo(partitionInfo)
+    wrapperTableInfo.setFactTable(tableSchema)
+    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
+    val thriftTable =
+      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+    thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
+      .setTime_stamp(System.currentTimeMillis)
+    carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
+      dbName, tableName, storePath)
+    CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
+    // update the schema modified time
+    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
+    // sparkSession.catalog.refreshTable(tableName)
+    Seq.empty
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+    val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
+    val tableName = model.tableName
+    var locks = List.empty[ICarbonLock]
+    var success = false
+    try {
+      val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+        LockUsage.COMPACTION_LOCK,
+        LockUsage.DELETE_SEGMENT_LOCK,
+        LockUsage.DROP_TABLE_LOCK,
+        LockUsage.CLEAN_FILES_LOCK,
+        LockUsage.ALTER_PARTITION_LOCK)
+      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
+        locksToBeAcquired)(sparkSession)
+      val carbonLoadModel = new CarbonLoadModel()
+      val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+      val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+      val table = relation.tableMeta.carbonTable
+      val dataLoadSchema = new CarbonDataLoadSchema(table)
+      // Need to fill dimension relation
+      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+      carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
+      carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
+      carbonLoadModel.setStorePath(relation.tableMeta.storePath)
+      val loadStartTime = CarbonUpdateUtil.readCurrentTime
+      carbonLoadModel.setFactTimeStamp(loadStartTime)
+      CarbonDataRDDFactory.alterTableDropPartition(
+        sparkSession.sqlContext,
+        model.partitionId,
+        carbonLoadModel,
+        model.dropWithData,
+        oldPartitionIds.asScala.toList
+      )
+      success = true
+    } catch {
+      case e: Exception =>
+        sys.error(s"Drop Partition failed. Please check logs for more info. ${ e.getMessage } ")
+        success = false
+    } finally {
+      CacheProvider.getInstance().dropAllCache()
+      AlterTableUtil.releaseLocks(locks)
+      LOGGER.info("Locks released after alter table drop partition action.")
+      LOGGER.audit("Locks released after alter table drop partition action.")
+    }
+    LOGGER.info(s"Alter table drop partition is successful for table $dbName.$tableName")
+    LOGGER.audit(s"Alter table drop partition is successful for table $dbName.$tableName")
+    Seq.empty
+  }
+}


[4/6] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by qi...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
deleted file mode 100644
index 7ed280e..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ /dev/null
@@ -1,1315 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command
-
-import java.text.SimpleDateFormat
-import java.util
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-import scala.language.implicitConversions
-
-import org.apache.commons.lang3.StringUtils
-import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
-import org.apache.spark.util.{AlterTableUtil, CausedBy, FileUtils, PartitionUtils}
-import org.codehaus.jackson.map.ObjectMapper
-
-import org.apache.carbondata.api.CarbonStore
-import org.apache.carbondata.common.constants.LoggerAction
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.CacheProvider
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.dictionary.server.DictionaryServer
-import org.apache.carbondata.core.exception.InvalidConfigurationException
-import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.PartitionInfo
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
-import org.apache.carbondata.core.metadata.schema.table.TableInfo
-import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.format
-import org.apache.carbondata.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.etl.DataLoadingException
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.newflow.exception.NoRetryException
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.load.ValidateUtil
-import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DictionaryLoadModel}
-import org.apache.carbondata.spark.util.{CarbonSparkUtil, CommonUtil, GlobalDictionaryUtil}
-
-object Checker {
-  def validateTableExists(
-      dbName: Option[String],
-      tableName: String,
-      session: SparkSession): Unit = {
-    val identifier = TableIdentifier(tableName, dbName)
-    if (!CarbonEnv.getInstance(session).carbonMetastore.tableExists(identifier)(session)) {
-      val err = s"table $dbName.$tableName not found"
-      LogServiceFactory.getLogService(this.getClass.getName).error(err)
-      throw new IllegalArgumentException(err)
-    }
-  }
-}
-
-/**
- * Interface for command that modifies schema
- */
-trait SchemaProcessCommand {
-  def processSchema(sparkSession: SparkSession): Seq[Row]
-}
-
-/**
- * Interface for command that need to process data in file system
- */
-trait DataProcessCommand {
-  def processData(sparkSession: SparkSession): Seq[Row]
-}
-
-/**
- * Command for show table partitions Command
- *
- * @param tableIdentifier
- */
-private[sql] case class ShowCarbonPartitionsCommand(
-    tableIdentifier: TableIdentifier) extends RunnableCommand with SchemaProcessCommand {
-
-  override val output = CommonUtil.partitionInfoOutput
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(tableIdentifier)(sparkSession).
-        asInstanceOf[CarbonRelation]
-    val carbonTable = relation.tableMeta.carbonTable
-    val tableName = carbonTable.getFactTableName
-    val partitionInfo = carbonTable.getPartitionInfo(
-      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
-    if (partitionInfo == null) {
-      throw new AnalysisException(
-        s"SHOW PARTITIONS is not allowed on a table that is not partitioned: $tableName")
-    }
-    val partitionType = partitionInfo.getPartitionType
-    val columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
-    val LOGGER = LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
-    LOGGER.info("partition column name:" + columnName)
-    CommonUtil.getPartitionInfo(columnName, partitionType, partitionInfo)
-  }
-}
-
-/**
- * Command for the compaction in alter table command
- *
- * @param alterTableModel
- */
-case class AlterTableCompaction(alterTableModel: AlterTableModel) extends RunnableCommand
-    with DataProcessCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableModel.tableName.toLowerCase
-    val databaseName = alterTableModel.dbName.getOrElse(sparkSession.catalog.currentDatabase)
-    val relation =
-      CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .lookupRelation(Option(databaseName), tableName)(sparkSession)
-          .asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      sys.error(s"Table $databaseName.$tableName does not exist")
-    }
-    if (null == relation.tableMeta.carbonTable) {
-      LOGGER.error(s"alter table failed. table not found: $databaseName.$tableName")
-      sys.error(s"alter table failed. table not found: $databaseName.$tableName")
-    }
-
-    val carbonLoadModel = new CarbonLoadModel()
-
-    val table = relation.tableMeta.carbonTable
-    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.carbonTable.getStorePath)
-
-    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,
-            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
-  }
-}
-
-/**
- * Command for Alter Table Add & Split partition
- * Add is a special case of Splitting the default partition (part0)
- * @param splitPartitionModel
- */
-case class AlterTableSplitPartitionCommand(splitPartitionModel: AlterTableSplitPartitionModel)
-  extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-  val tableName = splitPartitionModel.tableName
-  val splitInfo = splitPartitionModel.splitInfo
-  val partitionId = splitPartitionModel.partitionId.toInt
-  var partitionInfo: PartitionInfo = null
-  var carbonMetaStore: CarbonMetaStore = null
-  var relation: CarbonRelation = null
-  var dbName: String = null
-  var storePath: String = null
-  var table: CarbonTable = null
-  var carbonTableIdentifier: CarbonTableIdentifier = null
-  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
-  val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
-    .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-  val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
-    .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-  val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-    LockUsage.COMPACTION_LOCK,
-    LockUsage.DELETE_SEGMENT_LOCK,
-    LockUsage.DROP_TABLE_LOCK,
-    LockUsage.CLEAN_FILES_LOCK,
-    LockUsage.ALTER_PARTITION_LOCK)
-
-  // TODO will add rollback function incase process data failure
-  def run(sparkSession: SparkSession): Seq[Row] = {
-      processSchema(sparkSession)
-      processData(sparkSession)
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    dbName = splitPartitionModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
-      .asInstanceOf[CarbonRelation]
-    carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
-    storePath = relation.tableMeta.storePath
-    if (relation == null) {
-      sys.error(s"Table $dbName.$tableName does not exist")
-    }
-    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
-    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
-      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
-      sys.error(s"Alter table failed. table not found: $dbName.$tableName")
-    }
-    table = relation.tableMeta.carbonTable
-    partitionInfo = table.getPartitionInfo(tableName)
-    val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
-    // keep a copy of partitionIdList before update partitionInfo.
-    // will be used in partition data scan
-    oldPartitionIds.addAll(partitionIds.asJava)
-
-    if (partitionInfo == null) {
-      sys.error(s"Table $tableName is not a partition table.")
-    }
-    if (partitionInfo.getPartitionType == PartitionType.HASH) {
-      sys.error(s"Hash partition table cannot be added or split!")
-    }
-    PartitionUtils.updatePartitionInfo(partitionInfo, partitionIds, partitionId,
-      splitInfo, timestampFormatter, dateFormatter)
-
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
-    val schemaFilePath = carbonTablePath.getSchemaFilePath
-    // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
-    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
-      dbName, tableName, storePath)
-    val tableSchema = wrapperTableInfo.getFactTable
-    tableSchema.setPartitionInfo(partitionInfo)
-    wrapperTableInfo.setFactTable(tableSchema)
-    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
-    val thriftTable =
-      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-    carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
-      dbName, tableName, storePath)
-    CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
-    // update the schema modified time
-    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
-    sparkSession.catalog.refreshTable(tableName)
-    Seq.empty
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    var locks = List.empty[ICarbonLock]
-    var success = false
-    try {
-      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
-        locksToBeAcquired)(sparkSession)
-      val carbonLoadModel = new CarbonLoadModel()
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-      carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
-      carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
-      carbonLoadModel.setStorePath(storePath)
-      val loadStartTime = CarbonUpdateUtil.readCurrentTime
-      carbonLoadModel.setFactTimeStamp(loadStartTime)
-      CarbonDataRDDFactory.alterTableSplitPartition(sparkSession.sqlContext,
-        partitionId.toString,
-        carbonLoadModel,
-        oldPartitionIds.asScala.toList
-      )
-      success = true
-    } catch {
-      case e: Exception =>
-        success = false
-        sys.error(s"Add/Split Partition failed. Please check logs for more info. ${ e.getMessage }")
-    } finally {
-      AlterTableUtil.releaseLocks(locks)
-      CacheProvider.getInstance().dropAllCache()
-      LOGGER.info("Locks released after alter table add/split partition action.")
-      LOGGER.audit("Locks released after alter table add/split partition action.")
-      if (success) {
-        LOGGER.info(s"Alter table add/split partition is successful for table $dbName.$tableName")
-        LOGGER.audit(s"Alter table add/split partition is successful for table $dbName.$tableName")
-      }
-    }
-    Seq.empty
-  }
-}
-
-case class AlterTableDropPartition(alterTableDropPartitionModel: AlterTableDropPartitionModel)
-  extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-  val tableName = alterTableDropPartitionModel.tableName
-  var dbName: String = null
-  val partitionId = alterTableDropPartitionModel.partitionId
-  val dropWithData = alterTableDropPartitionModel.dropWithData
-  if (partitionId == 0 ) {
-    sys.error(s"Cannot drop default partition! Please use delete statement!")
-  }
-  var partitionInfo: PartitionInfo = null
-  var carbonMetaStore: CarbonMetaStore = null
-  var relation: CarbonRelation = null
-  var storePath: String = null
-  var table: CarbonTable = null
-  var carbonTableIdentifier: CarbonTableIdentifier = null
-  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
-  val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-    LockUsage.COMPACTION_LOCK,
-    LockUsage.DELETE_SEGMENT_LOCK,
-    LockUsage.DROP_TABLE_LOCK,
-    LockUsage.CLEAN_FILES_LOCK,
-    LockUsage.ALTER_PARTITION_LOCK)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-    processData(sparkSession)
-    Seq.empty
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    dbName = alterTableDropPartitionModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
-      .asInstanceOf[CarbonRelation]
-    carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
-    storePath = relation.tableMeta.storePath
-    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
-    if (relation == null) {
-      sys.error(s"Table $dbName.$tableName does not exist")
-    }
-    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
-      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
-      sys.error(s"Alter table failed. table not found: $dbName.$tableName")
-    }
-    table = relation.tableMeta.carbonTable
-    partitionInfo = table.getPartitionInfo(tableName)
-    if (partitionInfo == null) {
-      sys.error(s"Table $tableName is not a partition table.")
-    }
-    val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
-    // keep a copy of partitionIdList before update partitionInfo.
-    // will be used in partition data scan
-    oldPartitionIds.addAll(partitionIds.asJava)
-    val partitionIndex = partitionIds.indexOf(Integer.valueOf(partitionId))
-    partitionInfo.getPartitionType match {
-      case PartitionType.HASH => sys.error(s"Hash partition cannot be dropped!")
-      case PartitionType.RANGE =>
-        val rangeInfo = new util.ArrayList(partitionInfo.getRangeInfo)
-        val rangeToRemove = partitionInfo.getRangeInfo.get(partitionIndex - 1)
-        rangeInfo.remove(rangeToRemove)
-        partitionInfo.setRangeInfo(rangeInfo)
-      case PartitionType.LIST =>
-        val listInfo = new util.ArrayList(partitionInfo.getListInfo)
-        val listToRemove = partitionInfo.getListInfo.get(partitionIndex - 1)
-        listInfo.remove(listToRemove)
-        partitionInfo.setListInfo(listInfo)
-      case PartitionType.RANGE_INTERVAL =>
-        sys.error(s"Dropping range interval partition isn't support yet!")
-    }
-    partitionInfo.dropPartition(partitionIndex)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
-    val schemaFilePath = carbonTablePath.getSchemaFilePath
-    // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
-
-    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
-      dbName, tableName, storePath)
-    val tableSchema = wrapperTableInfo.getFactTable
-    tableSchema.setPartitionInfo(partitionInfo)
-    wrapperTableInfo.setFactTable(tableSchema)
-    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
-    val thriftTable =
-      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-    thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-      .setTime_stamp(System.currentTimeMillis)
-    carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
-      dbName, tableName, storePath)
-    CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
-    // update the schema modified time
-    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
-    // sparkSession.catalog.refreshTable(tableName)
-    Seq.empty
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    var locks = List.empty[ICarbonLock]
-    var success = false
-    try {
-      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
-        locksToBeAcquired)(sparkSession)
-      val carbonLoadModel = new CarbonLoadModel()
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      // Need to fill dimension relation
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-      carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
-      carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
-      carbonLoadModel.setStorePath(storePath)
-      val loadStartTime = CarbonUpdateUtil.readCurrentTime
-      carbonLoadModel.setFactTimeStamp(loadStartTime)
-      CarbonDataRDDFactory.alterTableDropPartition(sparkSession.sqlContext,
-        partitionId,
-        carbonLoadModel,
-        dropWithData,
-        oldPartitionIds.asScala.toList
-      )
-      success = true
-    } catch {
-      case e: Exception =>
-        sys.error(s"Drop Partition failed. Please check logs for more info. ${ e.getMessage } ")
-      success = false
-    } finally {
-      CacheProvider.getInstance().dropAllCache()
-      AlterTableUtil.releaseLocks(locks)
-      LOGGER.info("Locks released after alter table drop partition action.")
-      LOGGER.audit("Locks released after alter table drop partition action.")
-    }
-    LOGGER.info(s"Alter table drop partition is successful for table $dbName.$tableName")
-    LOGGER.audit(s"Alter table drop partition is successful for table $dbName.$tableName")
-    Seq.empty
-  }
-}
-
-  case class CreateTable(cm: TableModel, createDSTable: Boolean = true) extends RunnableCommand
-    with SchemaProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-  }
-
-  def setV(ref: Any, name: String, value: Any): Unit = {
-    ref.getClass.getFields.find(_.getName == name).get
-      .set(ref, value.asInstanceOf[AnyRef])
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val storePath = CarbonEnv.getInstance(sparkSession).storePath
-    CarbonEnv.getInstance(sparkSession).carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(storePath)
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    cm.databaseName = getDB.getDatabaseName(cm.databaseNameOp, sparkSession)
-    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)
-
-    // Add validation for sort scope when create table
-    val sortScope = tableInfo.getFactTable.getTableProperties
-      .getOrDefault("sort_scope", CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT)
-    if (!CarbonUtil.isValidSortOption(sortScope)) {
-      throw new InvalidConfigurationException(s"Passing invalid SORT_SCOPE '$sortScope'," +
-        s" valid SORT_SCOPE are 'NO_SORT', 'BATCH_SORT', 'LOCAL_SORT' and 'GLOBAL_SORT' ")
-    }
-
-    if (tableInfo.getFactTable.getListOfColumns.size <= 0) {
-      sys.error("No Dimensions found. Table should have at least one dimesnion !")
-    }
-
-    if (sparkSession.sessionState.catalog.listTables(dbName)
-        .exists(_.table.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 {
-      val tableIdentifier = AbsoluteTableIdentifier.from(storePath, dbName, tbName)
-      // Add Database to catalog and persist
-      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      val tablePath = tableIdentifier.getTablePath
-      val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tablePath)
-      if (createDSTable) {
-        try {
-          val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
-          cm.dimCols.foreach(f => fields(f.schemaOrdinal) = f)
-          cm.msrCols.foreach(f => fields(f.schemaOrdinal) = f)
-
-          sparkSession.sql(
-            s"""CREATE TABLE $dbName.$tbName
-               |(${ fields.map(f => f.rawSchema).mkString(",") })
-               |USING org.apache.spark.sql.CarbonSource""".stripMargin +
-                s""" OPTIONS (tableName "$tbName", dbName "$dbName", tablePath """.stripMargin +
-                s""""$tablePath"$carbonSchemaString) """)
-        } catch {
-          case e: Exception =>
-            val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
-            // call the drop table to delete the created table.
-            CarbonEnv.getInstance(sparkSession).carbonMetastore
-                .dropTable(tablePath, 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
-  }
-}
-
-case class DeleteLoadsById(
-    loadids: Seq[String],
-    databaseNameOp: Option[String],
-    tableName: String) extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
-    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
-        lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
-        tableMeta.carbonTable
-    CarbonStore.deleteLoadById(
-      loadids,
-      getDB.getDatabaseName(databaseNameOp, sparkSession),
-      tableName,
-      carbonTable
-    )
-    Seq.empty
-  }
-}
-
-case class DeleteLoadsByLoadDate(
-    databaseNameOp: Option[String],
-    tableName: String,
-    dateField: String,
-    loadDate: String) extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
-    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
-        lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
-        tableMeta.carbonTable
-    CarbonStore.deleteLoadByDate(
-      loadDate,
-      getDB.getDatabaseName(databaseNameOp, sparkSession),
-      tableName,
-      carbonTable
-    )
-    Seq.empty
-  }
-}
-
-object LoadTable {
-
-  def updateTableMetadata(carbonLoadModel: CarbonLoadModel,
-      sqlContext: SQLContext,
-      model: DictionaryLoadModel,
-      noDictDimension: Array[CarbonDimension]): Unit = {
-    val sparkSession = sqlContext.sparkSession
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(model.hdfsLocation,
-      model.table)
-
-    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    // read TableInfo
-    val tableInfo: format.TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-
-    // modify TableInfo
-    val columns = tableInfo.getFact_table.getTable_columns
-    for (i <- 0 until columns.size) {
-      if (noDictDimension.exists(x => columns.get(i).getColumn_id.equals(x.getColumnId))) {
-        columns.get(i).encoders.remove(org.apache.carbondata.format.Encoding.DICTIONARY)
-      }
-    }
-    val entry = tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-    entry.setTime_stamp(System.currentTimeMillis())
-
-    // write TableInfo
-    metastore.updateTableSchema(carbonTablePath.getCarbonTableIdentifier,
-      carbonTablePath.getCarbonTableIdentifier,
-      tableInfo, entry, carbonTablePath.getPath)(sparkSession)
-
-    // update the schema modified time
-    metastore.updateAndTouchSchemasUpdatedTime(model.hdfsLocation)
-
-    // update CarbonDataLoadSchema
-    val carbonTable = metastore.lookupRelation(Option(model.table.getDatabaseName),
-      model.table.getTableName)(sqlContext.sparkSession).asInstanceOf[CarbonRelation].tableMeta
-      .carbonTable
-    carbonLoadModel.setCarbonDataLoadSchema(new CarbonDataLoadSchema(carbonTable))
-  }
-
-}
-
-case class LoadTableByInsert(relation: CarbonDatasourceHadoopRelation,
-    child: LogicalPlan,
-    overwrite: Boolean)
-  extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val df = Dataset.ofRows(sparkSession, child)
-    val header = relation.tableSchema.get.fields.map(_.name).mkString(",")
-    val load = LoadTable(
-      Some(relation.carbonRelation.databaseName),
-      relation.carbonRelation.tableName,
-      null,
-      Seq(),
-      scala.collection.immutable.Map("fileheader" -> header),
-      overwrite,
-      null,
-      Some(df)).run(sparkSession)
-    // updating relation metadata. This is in case of auto detect high cardinality
-    relation.carbonRelation.metaData =
-        CarbonSparkUtil.createSparkMeta(relation.carbonRelation.tableMeta.carbonTable)
-    load
-  }
-}
-
-case class LoadTable(
-    databaseNameOp: Option[String],
-    tableName: String,
-    factPathFromUser: String,
-    dimFilesPath: Seq[DataLoadTableFileMapping],
-    options: scala.collection.immutable.Map[String, String],
-    isOverwriteTable: Boolean,
-    var inputSqlString: String = null,
-    dataFrame: Option[DataFrame] = None,
-    updateModel: Option[UpdateTableModel] = None) extends RunnableCommand with DataProcessCommand {
-
-  private def getFinalOptions(carbonProperty: CarbonProperties):
-  scala.collection.mutable.Map[String, String] = {
-    var optionsFinal = scala.collection.mutable.Map[String, String]()
-    optionsFinal.put("delimiter", options.getOrElse("delimiter", ","))
-    optionsFinal.put("quotechar", options.getOrElse("quotechar", "\""))
-    optionsFinal.put("fileheader", options.getOrElse("fileheader", ""))
-    optionsFinal.put("escapechar", options.getOrElse("escapechar", "\\"))
-    optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
-    optionsFinal.put("columndict", options.getOrElse("columndict", null))
-    optionsFinal
-      .put("serialization_null_format", options.getOrElse("serialization_null_format", "\\N"))
-    optionsFinal.put("bad_records_logger_enable", options.getOrElse("bad_records_logger_enable",
-      carbonProperty
-        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE,
-          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT)))
-    val badRecordActionValue = carbonProperty
-      .getProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
-        CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT)
-    optionsFinal.put("bad_records_action", options.getOrElse("bad_records_action", carbonProperty
-      .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION,
-        badRecordActionValue)))
-    optionsFinal
-      .put("is_empty_data_bad_record", options.getOrElse("is_empty_data_bad_record", carbonProperty
-        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD,
-          CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT)))
-    optionsFinal.put("all_dictionary_path", options.getOrElse("all_dictionary_path", ""))
-    optionsFinal
-      .put("complex_delimiter_level_1", options.getOrElse("complex_delimiter_level_1", "\\$"))
-    optionsFinal
-      .put("complex_delimiter_level_2", options.getOrElse("complex_delimiter_level_2", "\\:"))
-    optionsFinal.put("dateformat", options.getOrElse("dateformat",
-      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT,
-        CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)))
-
-    optionsFinal.put("global_sort_partitions", options.getOrElse("global_sort_partitions",
-      carbonProperty
-        .getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS, null)))
-
-    optionsFinal.put("maxcolumns", options.getOrElse("maxcolumns", null))
-
-    optionsFinal.put("batch_sort_size_inmb", options.getOrElse("batch_sort_size_inmb",
-      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB,
-        carbonProperty.getProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
-          CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))))
-
-    optionsFinal.put("bad_record_path", options.getOrElse("bad_record_path",
-      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
-        carbonProperty.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC,
-          CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))))
-
-    val useOnePass = options.getOrElse("single_pass",
-      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
-        CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT)).trim.toLowerCase match {
-      case "true" =>
-        true
-      case "false" =>
-        // when single_pass = false  and if either alldictionarypath
-        // or columnDict is configured the do not allow load
-        if (StringUtils.isNotEmpty(optionsFinal("all_dictionary_path")) ||
-            StringUtils.isNotEmpty(optionsFinal("columndict"))) {
-          throw new MalformedCarbonCommandException(
-            "Can not use all_dictionary_path or columndict without single_pass.")
-        } else {
-          false
-        }
-      case illegal =>
-        val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-        LOGGER.error(s"Can't use single_pass, because illegal syntax found: [" + illegal + "] " +
-                     "Please set it as 'true' or 'false'")
-        false
-    }
-    optionsFinal.put("single_pass", useOnePass.toString)
-    optionsFinal
-  }
-
-  private def checkDefaultValue(value: String, default: String) = {
-    if (StringUtils.isEmpty(value)) {
-      default
-    } else {
-      value
-    }
-  }
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    if (dataFrame.isDefined && updateModel.isEmpty) {
-      val rdd = dataFrame.get.rdd
-      if (rdd.partitions == null || rdd.partitions.length == 0) {
-        LOGGER.warn("DataLoading finished. No data was loaded.")
-        return Seq.empty
-      }
-    }
-
-    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
-
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(Option(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      sys.error(s"Table $dbName.$tableName does not exist")
-    }
-    if (null == relation.tableMeta.carbonTable) {
-      LOGGER.error(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 carbonProperty: CarbonProperties = CarbonProperties.getInstance()
-    carbonProperty.addProperty("zookeeper.enable.lock", "false")
-    val optionsFinal = getFinalOptions(carbonProperty)
-
-    val tableProperties = relation.tableMeta.carbonTable.getTableInfo
-      .getFactTable.getTableProperties
-
-    optionsFinal.put("sort_scope", tableProperties.getOrDefault("sort_scope",
-        carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
-          carbonProperty.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
-            CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))))
-
-    try {
-      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.carbonTable.getStorePath)
-
-      val table = relation.tableMeta.carbonTable
-      carbonLoadModel.setTableName(table.getFactTableName)
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      // Need to fill dimension relation
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-
-      val 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 sort_scope = optionsFinal("sort_scope")
-      val single_pass = optionsFinal("single_pass")
-      val bad_records_logger_enable = optionsFinal("bad_records_logger_enable")
-      val bad_records_action = optionsFinal("bad_records_action")
-      val bad_record_path = optionsFinal("bad_record_path")
-      val global_sort_partitions = optionsFinal("global_sort_partitions")
-      val dateFormat = optionsFinal("dateformat")
-      val delimeter = optionsFinal("delimiter")
-      val complex_delimeter_level1 = optionsFinal("complex_delimiter_level_1")
-      val complex_delimeter_level2 = optionsFinal("complex_delimiter_level_2")
-      val all_dictionary_path = optionsFinal("all_dictionary_path")
-      val column_dict = optionsFinal("columndict")
-      ValidateUtil.validateDateFormat(dateFormat, table, tableName)
-      ValidateUtil.validateSortScope(table, sort_scope)
-
-      if (bad_records_logger_enable.toBoolean ||
-          LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
-        if (!CarbonUtil.isValidBadStorePath(bad_record_path)) {
-          sys.error("Invalid bad records location.")
-        }
-      }
-      carbonLoadModel.setBadRecordsLocation(bad_record_path)
-
-      ValidateUtil.validateGlobalSortPartitions(global_sort_partitions)
-      carbonLoadModel.setEscapeChar(checkDefaultValue(optionsFinal("escapechar"), "\\"))
-      carbonLoadModel.setQuoteChar(checkDefaultValue(optionsFinal("quotechar"), "\""))
-      carbonLoadModel.setCommentChar(checkDefaultValue(optionsFinal("commentchar"), "#"))
-
-      // if there isn't file header in csv file and load sql doesn't provide FILEHEADER option,
-      // we should use table schema to generate file header.
-      var fileHeader = optionsFinal("fileheader")
-      val headerOption = options.get("header")
-      if (headerOption.isDefined) {
-        // whether the csv file has file header
-        // the default value is true
-        val header = try {
-          headerOption.get.toBoolean
-        } catch {
-          case ex: IllegalArgumentException =>
-            throw new MalformedCarbonCommandException(
-              "'header' option should be either 'true' or 'false'. " + ex.getMessage)
-        }
-        header match {
-          case true =>
-            if (fileHeader.nonEmpty) {
-              throw new MalformedCarbonCommandException(
-                "When 'header' option is true, 'fileheader' option is not required.")
-            }
-          case false =>
-            // generate file header
-            if (fileHeader.isEmpty) {
-              fileHeader = table.getCreateOrderColumn(table.getFactTableName)
-                  .asScala.map(_.getColName).mkString(",")
-            }
-        }
-      }
-
-      carbonLoadModel.setDateFormat(dateFormat)
-      carbonLoadModel.setDefaultTimestampFormat(carbonProperty.getProperty(
-        CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-      carbonLoadModel.setDefaultDateFormat(carbonProperty.getProperty(
-        CarbonCommonConstants.CARBON_DATE_FORMAT,
-        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-      carbonLoadModel
-          .setSerializationNullFormat(
-            TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + "," +
-                optionsFinal("serialization_null_format"))
-      carbonLoadModel
-          .setBadRecordsLoggerEnable(
-            TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName + "," + bad_records_logger_enable)
-      carbonLoadModel
-          .setBadRecordsAction(
-            TableOptionConstant.BAD_RECORDS_ACTION.getName + "," + bad_records_action)
-      carbonLoadModel
-          .setIsEmptyDataBadRecord(
-            DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," +
-                optionsFinal("is_empty_data_bad_record"))
-      carbonLoadModel.setSortScope(sort_scope)
-      carbonLoadModel.setBatchSortSizeInMb(optionsFinal("batch_sort_size_inmb"))
-      carbonLoadModel.setGlobalSortPartitions(global_sort_partitions)
-      carbonLoadModel.setUseOnePass(single_pass.toBoolean)
-      if (delimeter.equalsIgnoreCase(complex_delimeter_level1) ||
-          complex_delimeter_level1.equalsIgnoreCase(complex_delimeter_level2) ||
-          delimeter.equalsIgnoreCase(complex_delimeter_level2)) {
-        sys.error(s"Field Delimiter & Complex types delimiter are same")
-      }
-      else {
-        carbonLoadModel.setComplexDelimiterLevel1(
-          CarbonUtil.delimiterConverter(complex_delimeter_level1))
-        carbonLoadModel.setComplexDelimiterLevel2(
-          CarbonUtil.delimiterConverter(complex_delimeter_level2))
-      }
-      // set local dictionary path, and dictionary file extension
-      carbonLoadModel.setAllDictPath(all_dictionary_path)
-
-      val 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(delimeter))
-        carbonLoadModel.setCsvHeader(fileHeader)
-        carbonLoadModel.setColDictFilePath(column_dict)
-        carbonLoadModel.setDirectLoad(true)
-        carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
-        val validatedMaxColumns = CommonUtil.validateMaxColumns(carbonLoadModel.getCsvHeaderColumns,
-          optionsFinal("maxcolumns"))
-        carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
-        GlobalDictionaryUtil.updateTableMetadataFunc = LoadTable.updateTableMetadata
-        val storePath = relation.tableMeta.storePath
-        // add the start entry for the new load in the table status file
-        if (!updateModel.isDefined) {
-          CommonUtil.
-            readAndUpdateLoadProgressInTableMeta(carbonLoadModel, storePath, isOverwriteTable)
-        }
-        if (isOverwriteTable) {
-          LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
-        }
-        if (null == carbonLoadModel.getLoadMetadataDetails) {
-          CommonUtil.readLoadMetadataDetails(carbonLoadModel)
-        }
-        if (carbonLoadModel.getLoadMetadataDetails.isEmpty && carbonLoadModel.getUseOnePass &&
-            StringUtils.isEmpty(column_dict) && StringUtils.isEmpty(all_dictionary_path)) {
-          LOGGER.info(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
-          LOGGER.audit(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
-          carbonLoadModel.setUseOnePass(false)
-        }
-        // Create table and metadata folders if not exist
-        val carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(storePath, table.getCarbonTableIdentifier)
-        val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
-        val fileType = FileFactory.getFileType(metadataDirectoryPath)
-        if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
-          FileFactory.mkdirs(metadataDirectoryPath, fileType)
-        }
-        if (carbonLoadModel.getUseOnePass) {
-          val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-          val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-              .getCarbonTableIdentifier
-          val carbonTablePath = CarbonStorePath
-              .getCarbonTablePath(storePath, carbonTableIdentifier)
-          val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
-          val dimensions = carbonTable.getDimensionByTableName(
-            carbonTable.getFactTableName).asScala.toArray
-          val colDictFilePath = carbonLoadModel.getColDictFilePath
-          if (!StringUtils.isEmpty(colDictFilePath)) {
-            carbonLoadModel.initPredefDictMap()
-            // generate predefined dictionary
-            GlobalDictionaryUtil
-                .generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
-                  dimensions, carbonLoadModel, sparkSession.sqlContext, storePath, dictFolderPath)
-          }
-          if (!StringUtils.isEmpty(all_dictionary_path)) {
-            carbonLoadModel.initPredefDictMap()
-            GlobalDictionaryUtil
-                .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
-                  carbonLoadModel,
-                  storePath,
-                  carbonTableIdentifier,
-                  dictFolderPath,
-                  dimensions,
-                  all_dictionary_path)
-          }
-          // dictionaryServerClient dictionary generator
-          val dictionaryServerPort = carbonProperty
-              .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-                CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
-          val sparkDriverHost = sparkSession.sqlContext.sparkContext.
-              getConf.get("spark.driver.host")
-          carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
-          // start dictionary server when use one pass load and dimension with DICTIONARY
-          // encoding is present.
-          val allDimensions = table.getAllDimensions.asScala.toList
-          val createDictionary = allDimensions.exists {
-            carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
-                !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
-          }
-          val server: Option[DictionaryServer] = if (createDictionary) {
-            val dictionaryServer = DictionaryServer
-              .getInstance(dictionaryServerPort.toInt, carbonTable)
-            carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-            sparkSession.sparkContext.addSparkListener(new SparkListener() {
-              override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-                dictionaryServer.shutdown()
-              }
-            })
-            Some(dictionaryServer)
-          } else {
-            None
-          }
-          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
-            carbonLoadModel,
-            relation.tableMeta.storePath,
-            columnar,
-            partitionStatus,
-            server,
-            isOverwriteTable,
-            dataFrame,
-            updateModel)
-        } else {
-          val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
-            val fields = dataFrame.get.schema.fields
-            import org.apache.spark.sql.functions.udf
-            // extracting only segment from tupleId
-            val getSegIdUDF = udf((tupleId: String) =>
-              CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID))
-            // getting all fields except tupleId field as it is not required in the value
-            var otherFields = fields.toSeq
-                .filter(field => !field.name
-                    .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
-                .map(field => new Column(field.name))
-
-            // extract tupleId field which will be used as a key
-            val segIdColumn = getSegIdUDF(new Column(UnresolvedAttribute
-                .quotedString(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))).
-              as(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID)
-            // use dataFrameWithoutTupleId as dictionaryDataFrame
-            val dataFrameWithoutTupleId = dataFrame.get.select(otherFields: _*)
-            otherFields = otherFields :+ segIdColumn
-            // use dataFrameWithTupleId as loadDataFrame
-            val dataFrameWithTupleId = dataFrame.get.select(otherFields: _*)
-            (Some(dataFrameWithoutTupleId), Some(dataFrameWithTupleId))
-          } else {
-            (dataFrame, dataFrame)
-          }
-
-            GlobalDictionaryUtil.generateGlobalDictionary(
-                sparkSession.sqlContext,
-                carbonLoadModel,
-                relation.tableMeta.storePath,
-                dictionaryDataFrame)
-          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
-            carbonLoadModel,
-            relation.tableMeta.storePath,
-            columnar,
-            partitionStatus,
-            None,
-            isOverwriteTable,
-            loadDataFrame,
-            updateModel)
-        }
-      } catch {
-        case CausedBy(ex: NoRetryException) =>
-          LOGGER.error(ex, s"Dataload failure for $dbName.$tableName")
-          throw new RuntimeException(s"Dataload failure for $dbName.$tableName, ${ex.getMessage}")
-        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
-    }
-    Seq.empty
-  }
-}
-
-case class CleanFiles(
-    databaseNameOp: Option[String],
-    tableName: String, forceTableClean: Boolean = false)
-  extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
-    if (forceTableClean) {
-      CarbonStore.cleanFiles(
-        getDB.getDatabaseName(databaseNameOp, sparkSession),
-        tableName,
-        CarbonEnv.getInstance(sparkSession).storePath,
-        null,
-        forceTableClean)
-    } else {
-      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      val relation = catalog
-        .lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation]
-      val carbonTable = relation.tableMeta.carbonTable
-      CarbonStore.cleanFiles(
-        getDB.getDatabaseName(databaseNameOp, sparkSession),
-        tableName,
-        relation.asInstanceOf[CarbonRelation].tableMeta.storePath,
-        carbonTable,
-        forceTableClean)
-    }
-    Seq.empty
-  }
-}
-
-case class ShowLoads(
-    databaseNameOp: Option[String],
-    tableName: String,
-    limit: Option[String],
-    override val output: Seq[Attribute]) extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
-    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
-        lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
-        tableMeta.carbonTable
-    CarbonStore.showSegments(
-      getDB.getDatabaseName(databaseNameOp, sparkSession),
-      tableName,
-      limit,
-      carbonTable.getMetaDataFilepath
-    )
-  }
-}
-
-case class CarbonDropTableCommand(ifExistsSet: Boolean,
-    databaseNameOp: Option[String],
-    tableName: String)
-  extends RunnableCommand with SchemaProcessCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-    processData(sparkSession)
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val dbName = getDB.getDatabaseName(databaseNameOp, sparkSession)
-    val identifier = TableIdentifier(tableName, Option(dbName))
-    val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName, "")
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
-    val carbonEnv = CarbonEnv.getInstance(sparkSession)
-    val catalog = carbonEnv.carbonMetastore
-    val storePath = carbonEnv.storePath
-    val tableIdentifier =
-      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath,
-        dbName.toLowerCase, tableName.toLowerCase)
-    catalog.checkSchemasModifiedTimeAndReloadTables(tableIdentifier.getStorePath)
-    val carbonLocks: scala.collection.mutable.ListBuffer[ICarbonLock] = ListBuffer()
-    try {
-       locksToBeAcquired foreach {
-        lock => carbonLocks += CarbonLockUtil.getLockObject(carbonTableIdentifier, lock)
-      }
-      LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
-
-      CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .dropTable(tableIdentifier.getTablePath, identifier)(sparkSession)
-      LOGGER.audit(s"Deleted table [$tableName] under database [$dbName]")
-    } catch {
-      case ex: Exception =>
-        LOGGER.error(ex, s"Dropping table $dbName.$tableName failed")
-        sys.error(s"Dropping table $dbName.$tableName failed: ${ex.getMessage}")
-    } finally {
-      if (carbonLocks.nonEmpty) {
-        val unlocked = carbonLocks.forall(_.unlock())
-        if (unlocked) {
-          logInfo("Table MetaData Unlocked Successfully")
-        }
-      }
-    }
-    Seq.empty
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    // delete the table folder
-    val dbName = getDB.getDatabaseName(databaseNameOp, sparkSession)
-    val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    val tableIdentifier =
-      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath, dbName, tableName)
-    val metadataFilePath =
-      CarbonStorePath.getCarbonTablePath(tableIdentifier).getMetadataDirectoryPath
-    val fileType = FileFactory.getFileType(metadataFilePath)
-    if (FileFactory.isFileExist(metadataFilePath, fileType)) {
-      val file = FileFactory.getCarbonFile(metadataFilePath, fileType)
-      CarbonUtil.deleteFoldersAndFilesSilent(file.getParentFile)
-    }
-    Seq.empty
-  }
-}
-
-private[sql] case class DescribeCommandFormatted(
-    child: SparkPlan,
-    override val output: Seq[Attribute],
-    tblIdentifier: TableIdentifier)
-  extends RunnableCommand with SchemaProcessCommand {
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-  }
-
-  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.foreach { x =>
-      results = results :+ (s"Column Group $index", x, "")
-      index = index + 1
-    }
-    results
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(tblIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
-    val mapper = new ObjectMapper()
-    val colProps = StringBuilder.newBuilder
-    val dims = relation.metaData.dims.map(x => x.toLowerCase)
-    var results: Seq[(String, String, String)] = child.schema.fields.map { field =>
-      val fieldName = field.name.toLowerCase
-      val comment = if (dims.contains(fieldName)) {
-        val dimension = relation.metaData.carbonTable.getDimensionByName(
-          relation.tableMeta.carbonTableIdentifier.getTableName, fieldName)
-        if (null != dimension.getColumnProperties && !dimension.getColumnProperties.isEmpty) {
-          colProps.append(fieldName).append(".")
-              .append(mapper.writeValueAsString(dimension.getColumnProperties))
-              .append(",")
-        }
-        if (dimension.hasEncoding(Encoding.DICTIONARY) &&
-            !dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          "DICTIONARY, KEY COLUMN" + (dimension.hasEncoding(Encoding.INVERTED_INDEX) match {
-            case false => ",NOINVERTEDINDEX"
-            case _ => ""
-          })
-        } else {
-          "KEY COLUMN" + (dimension.hasEncoding(Encoding.INVERTED_INDEX) match {
-            case false => ",NOINVERTEDINDEX"
-            case _ => ""
-          })
-        }
-      } 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(("SORT_SCOPE", carbonTable.getTableInfo.getFactTable
-      .getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants
-      .LOAD_SORT_SCOPE_DEFAULT), CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
-    results ++= Seq(("", "", ""), ("##Detailed Column property", "", ""))
-    if (colPropStr.length() > 0) {
-      results ++= Seq((colPropStr, "", ""))
-    } else {
-      results ++= Seq(("ADAPTIVE", "", ""))
-    }
-    results ++= Seq(("SORT_COLUMNS", relation.metaData.carbonTable.getSortColumns(
-      relation.tableMeta.carbonTableIdentifier.getTableName).asScala
-        .map(column => column).mkString(","), ""))
-    val dimension = carbonTable
-        .getDimensionByTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
-    results ++= getColumnGroups(dimension.asScala.toList)
-    if (carbonTable.getPartitionInfo(carbonTable.getFactTableName) != null) {
-      results ++=
-          Seq(("Partition Columns: ", carbonTable.getPartitionInfo(carbonTable.getFactTableName)
-              .getColumnSchemaList.asScala.map(_.getColumnName).mkString(","), ""))
-    }
-    results.map { case (name, dataType, comment) =>
-      Row(f"$name%-36s", f"$dataType%-80s", f"$comment%-72s")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
new file mode 100644
index 0000000..28c53a1
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
@@ -0,0 +1,90 @@
+/*
+ * 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.management
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableModel, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+
+/**
+ * Command for the compaction in alter table command
+ */
+case class AlterTableCompactionCommand(
+    alterTableModel: AlterTableModel)
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+    val tableName = alterTableModel.tableName.toLowerCase
+    val databaseName = alterTableModel.dbName.getOrElse(sparkSession.catalog.currentDatabase)
+    val relation =
+      CarbonEnv.getInstance(sparkSession).carbonMetastore
+        .lookupRelation(Option(databaseName), tableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      sys.error(s"Table $databaseName.$tableName does not exist")
+    }
+    if (null == relation.tableMeta.carbonTable) {
+      LOGGER.error(s"alter table failed. table not found: $databaseName.$tableName")
+      sys.error(s"alter table failed. table not found: $databaseName.$tableName")
+    }
+
+    val carbonLoadModel = new CarbonLoadModel()
+
+    val table = relation.tableMeta.carbonTable
+    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.carbonTable.getStorePath)
+
+    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,
+          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
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
new file mode 100644
index 0000000..2003bb1
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
@@ -0,0 +1,50 @@
+/*
+ * 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.management
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.api.CarbonStore
+
+case class CarbonShowLoadsCommand(
+    databaseNameOp: Option[String],
+    tableName: String,
+    limit: Option[String],
+    override val output: Seq[Attribute])
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
+    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
+      tableMeta.carbonTable
+    CarbonStore.showSegments(
+      GetDB.getDatabaseName(databaseNameOp, sparkSession),
+      tableName,
+      limit,
+      carbonTable.getMetaDataFilepath
+    )
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
new file mode 100644
index 0000000..9406335
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.management
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.api.CarbonStore
+
+case class CleanFilesCommand(
+    databaseNameOp: Option[String],
+    tableName: String, forceTableClean: Boolean = false)
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
+    if (forceTableClean) {
+      CarbonStore.cleanFiles(
+        GetDB.getDatabaseName(databaseNameOp, sparkSession),
+        tableName,
+        CarbonEnv.getInstance(sparkSession).storePath,
+        null,
+        forceTableClean)
+    } else {
+      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val relation = catalog
+        .lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation]
+      val carbonTable = relation.tableMeta.carbonTable
+      CarbonStore.cleanFiles(
+        GetDB.getDatabaseName(databaseNameOp, sparkSession),
+        tableName,
+        relation.asInstanceOf[CarbonRelation].tableMeta.storePath,
+        carbonTable,
+        forceTableClean)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
new file mode 100644
index 0000000..1ea4508
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.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.spark.sql.execution.command.management
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.api.CarbonStore
+
+case class DeleteLoadByIdCommand(
+    loadIds: Seq[String],
+    databaseNameOp: Option[String],
+    tableName: String) extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
+    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
+      tableMeta.carbonTable
+    CarbonStore.deleteLoadById(
+      loadIds,
+      GetDB.getDatabaseName(databaseNameOp, sparkSession),
+      tableName,
+      carbonTable
+    )
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
new file mode 100644
index 0000000..3d06b18
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
@@ -0,0 +1,50 @@
+/*
+ * 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.management
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.api.CarbonStore
+
+case class DeleteLoadByLoadDateCommand(
+    databaseNameOp: Option[String],
+    tableName: String,
+    dateField: String,
+    loadDate: String)
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
+    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
+      tableMeta.carbonTable
+    CarbonStore.deleteLoadByDate(
+      loadDate,
+      GetDB.getDatabaseName(databaseNameOp, sparkSession),
+      tableName,
+      carbonTable
+    )
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableByInsertCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableByInsertCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableByInsertCommand.scala
new file mode 100644
index 0000000..3f0e093
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableByInsertCommand.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.management
+
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Dataset, Row, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.command.{DataProcessCommand, RunnableCommand}
+
+import org.apache.carbondata.spark.util.CarbonSparkUtil
+
+case class LoadTableByInsertCommand(
+    relation: CarbonDatasourceHadoopRelation,
+    child: LogicalPlan,
+    overwrite: Boolean)
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val df = Dataset.ofRows(sparkSession, child)
+    val header = relation.tableSchema.get.fields.map(_.name).mkString(",")
+    val load = LoadTableCommand(
+      Some(relation.carbonRelation.databaseName),
+      relation.carbonRelation.tableName,
+      null,
+      Seq(),
+      scala.collection.immutable.Map("fileheader" -> header),
+      overwrite,
+      null,
+      Some(df)).run(sparkSession)
+    // updating relation metadata. This is in case of auto detect high cardinality
+    relation.carbonRelation.metaData =
+      CarbonSparkUtil.createSparkMeta(relation.carbonRelation.tableMeta.carbonTable)
+    load
+  }
+}


[6/6] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by qi...@apache.org.
[CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

This closes #1379


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

Branch: refs/heads/master
Commit: 6627cac0ccdfa8cc94cda188f211112e82ec3294
Parents: 2ec69f6
Author: Jacky Li <ja...@qq.com>
Authored: Sun Sep 24 23:43:20 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Wed Sep 27 14:32:20 2017 +0800

----------------------------------------------------------------------
 .../spark/sql/CarbonCatalystOperators.scala     |   16 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |    4 +-
 .../sql/CarbonDatasourceHadoopRelation.scala    |    4 +-
 .../org/apache/spark/sql/CarbonSource.scala     |    2 +-
 .../spark/sql/SparkUnknownExpression.scala      |    1 -
 .../execution/CarbonLateDecodeStrategy.scala    |  553 --------
 .../execution/command/AlterTableCommands.scala  |  460 ------
 .../command/CarbonCreateTableCommand.scala      |  107 ++
 .../CarbonDescribeFormattedCommand.scala        |  133 ++
 .../command/CarbonDropTableCommand.scala        |   94 ++
 .../sql/execution/command/DDLStrategy.scala     |  143 --
 .../sql/execution/command/IUDCommands.scala     |  857 ------------
 .../execution/command/carbonTableSchema.scala   | 1315 ------------------
 .../AlterTableCompactionCommand.scala           |   90 ++
 .../management/CarbonShowLoadsCommand.scala     |   50 +
 .../command/management/CleanFilesCommand.scala  |   58 +
 .../management/DeleteLoadByIdCommand.scala      |   48 +
 .../DeleteLoadByLoadDateCommand.scala           |   50 +
 .../management/LoadTableByInsertCommand.scala   |   53 +
 .../command/management/LoadTableCommand.scala   |  520 +++++++
 .../command/mutation/DeleteExecution.scala      |  322 +++++
 .../command/mutation/HorizontalCompaction.scala |  246 ++++
 .../HorizontalCompactionException.scala         |   24 +
 .../mutation/ProjectForDeleteCommand.scala      |  105 ++
 .../mutation/ProjectForUpdateCommand.scala      |  228 +++
 .../spark/sql/execution/command/package.scala   |   53 +
 .../AlterTableDropCarbonPartitionCommand.scala  |  176 +++
 .../AlterTableSplitCarbonPartitionCommand.scala |  187 +++
 .../partition/ShowCarbonPartitionsCommand.scala |   60 +
 .../schema/AlterTableAddColumnCommand.scala     |  115 ++
 .../AlterTableDataTypeChangeCommand.scala       |  116 ++
 .../schema/AlterTableDropColumnCommand.scala    |  148 ++
 .../schema/AlterTableRenameTableCommand.scala   |  174 +++
 .../strategy/CarbonLateDecodeStrategy.scala     |  554 ++++++++
 .../sql/execution/strategy/DDLStrategy.scala    |  162 +++
 .../spark/sql/hive/CarbonAnalysisRules.scala    |   10 +-
 .../spark/sql/hive/CarbonSessionState.scala     |    7 +-
 .../execution/command/CarbonHiveCommands.scala  |    5 +-
 .../sql/optimizer/CarbonLateDecodeRule.scala    |    3 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   41 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala |    4 +-
 .../org/apache/spark/util/Compaction.scala      |    5 +-
 .../org/apache/spark/util/TableLoader.scala     |    4 +-
 .../vectorreader/VectorReaderTestCase.scala     |    2 +-
 44 files changed, 3928 insertions(+), 3381 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
index 0c3414a..6815629 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -65,7 +65,7 @@ case class IncludeProfile(attributes: Seq[Attribute]) extends CarbonProfile(attr
 
 case class ExcludeProfile(attributes: Seq[Attribute]) extends CarbonProfile(attributes)
 
-object getDB {
+object GetDB {
 
   def getDatabaseName(dbName: Option[String], sparkSession: SparkSession): String = {
     dbName.getOrElse(
@@ -76,7 +76,10 @@ object getDB {
 /**
  * Shows Loads in a table
  */
-case class ShowLoadsCommand(databaseNameOp: Option[String], table: String, limit: Option[String])
+case class ShowLoadsCommand(
+    databaseNameOp: Option[String],
+    table: String,
+    limit: Option[String])
   extends Command {
 
   override def output: Seq[Attribute] = {
@@ -111,15 +114,6 @@ case class DeleteRecords(
 }
 
 /**
- * Describe formatted for hive table
- */
-case class DescribeFormattedCommand(sql: String, tblIdentifier: TableIdentifier) extends Command {
-
-  override def output: Seq[AttributeReference] =
-    Seq(AttributeReference("result", StringType, nullable = false)())
-}
-
-/**
  * A logical plan representing insertion into Hive table
  * This plan ignores nullability of ArrayType, MapType, StructType unlike InsertIntoTable
  * because Hive Table doesn't have nullability for ARRAY, MAP,STRUCT types.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
index e9f2758..a12d86b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql
 
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.compress.GzipCodec
-import org.apache.spark.sql.execution.command.LoadTable
+import org.apache.spark.sql.execution.command.management.LoadTableCommand
 import org.apache.spark.sql.types._
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -130,7 +130,7 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
    */
   private def loadDataFrame(options: CarbonOption): Unit = {
     val header = dataFrame.columns.mkString(",")
-    LoadTable(
+    LoadTableCommand(
       Some(options.dbName),
       options.tableName,
       null,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
index 4f4515d..d5adc2f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.CarbonInputMetrics
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.execution.command.LoadTableByInsert
+import org.apache.spark.sql.execution.command.management.LoadTableByInsertCommand
 import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation}
@@ -95,7 +95,7 @@ case class CarbonDatasourceHadoopRelation(
           CarbonCommonConstants.DEFAULT_MAX_NUMBER_OF_COLUMNS)
     }
     if (data.logicalPlan.output.size >= carbonRelation.output.size) {
-      LoadTableByInsert(this, data.logicalPlan, overwrite).run(sparkSession)
+      LoadTableByInsertCommand(this, data.logicalPlan, overwrite).run(sparkSession)
     } else {
       sys.error("Cannot insert into target table because column number are different")
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
index 1b021b0..f4f8b75 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -24,8 +24,8 @@ import org.apache.commons.lang.StringUtils
 import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
-import org.apache.spark.sql.execution.CarbonLateDecodeStrategy
 import org.apache.spark.sql.execution.command.{TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.strategy.CarbonLateDecodeStrategy
 import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
 import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
index 50d7dba..d536746 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
@@ -21,7 +21,6 @@ import java.util.{ArrayList, List}
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.CarbonBoundReference
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression, GenericInternalRow}
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
deleted file mode 100644
index 4d8e7ac..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
+++ /dev/null
@@ -1,553 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.expressions.{Attribute, _}
-import org.apache.spark.sql.catalyst.planning.PhysicalOperation
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.optimizer.CarbonDecoderRelation
-import org.apache.spark.sql.sources.{BaseRelation, Filter}
-import org.apache.spark.sql.types.{AtomicType, IntegerType, StringType}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.schema.BucketingInfo
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.CarbonAliasDecoderRelation
-import org.apache.carbondata.spark.rdd.CarbonScanRDD
-import org.apache.carbondata.spark.util.CarbonScalaUtil
-
-/**
- * Carbon specific optimization 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 {
-  val PUSHED_FILTERS = "PushedFilters"
-
-  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
-    plan match {
-      case PhysicalOperation(projects, filters, l: LogicalRelation)
-        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        val relation = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-        pruneFilterProject(
-          l,
-          projects,
-          filters,
-          (a, f, needDecoder) => toCatalystRDD(l, a, relation.buildScan(
-            a.map(_.name).toArray, f), needDecoder)) :: Nil
-      case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
-        if ((profile.isInstanceOf[IncludeProfile] && profile.isEmpty) ||
-            !CarbonDictionaryDecoder.
-              isRequiredToDecode(CarbonDictionaryDecoder.
-                getDictionaryColumnMapping(child.output, relations, profile, aliasMap))) {
-          planLater(child) :: Nil
-        } else {
-          CarbonDictionaryDecoder(relations,
-            profile,
-            aliasMap,
-            planLater(child),
-            SparkSession.getActiveSession.get
-          ) :: Nil
-        }
-      case _ => Nil
-    }
-  }
-
-
-  def getDecoderRDD(
-      logicalRelation: LogicalRelation,
-      projectExprsNeedToDecode: ArrayBuffer[AttributeReference],
-      rdd: RDD[InternalRow],
-      output: Seq[Attribute]): RDD[InternalRow] = {
-    val table = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-    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, Option(table.carbonRelation.tableName))
-      relation.addAttribute(newAttr)
-      newAttr
-    }
-
-    new CarbonDecoderRDD(
-      Seq(relation),
-      IncludeProfile(attrs),
-      CarbonAliasDecoderRelation(),
-      rdd,
-      output,
-      CarbonEnv.getInstance(SparkSession.getActiveSession.get).storePath,
-      table.carbonTable.getTableInfo.serialize())
-  }
-
-  private[this] def toCatalystRDD(
-      relation: LogicalRelation,
-      output: Seq[Attribute],
-      rdd: RDD[InternalRow],
-      needDecode: ArrayBuffer[AttributeReference]):
-  RDD[InternalRow] = {
-    if (needDecode.nonEmpty) {
-      rdd.asInstanceOf[CarbonScanRDD].setVectorReaderSupport(false)
-      getDecoderRDD(relation, needDecode, rdd, output)
-    } else {
-      rdd.asInstanceOf[CarbonScanRDD]
-        .setVectorReaderSupport(supportBatchedDataSource(relation.relation.sqlContext, output))
-      rdd
-    }
-  }
-
-  protected def pruneFilterProject(
-      relation: LogicalRelation,
-      projects: Seq[NamedExpression],
-      filterPredicates: Seq[Expression],
-      scanBuilder: (Seq[Attribute], Array[Filter],
-        ArrayBuffer[AttributeReference]) => RDD[InternalRow]) = {
-    pruneFilterProjectRaw(
-      relation,
-      projects,
-      filterPredicates,
-      (requestedColumns, _, pushedFilters, a) => {
-        scanBuilder(requestedColumns, pushedFilters.toArray, a)
-      })
-  }
-
-  protected def pruneFilterProjectRaw(
-      relation: LogicalRelation,
-      rawProjects: Seq[NamedExpression],
-      filterPredicates: Seq[Expression],
-      scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
-        ArrayBuffer[AttributeReference]) => RDD[InternalRow]) = {
-    val projects = rawProjects.map {p =>
-      p.transform {
-        case CustomDeterministicExpression(exp) => exp
-      }
-    }.asInstanceOf[Seq[NamedExpression]]
-
-    val projectSet = AttributeSet(projects.flatMap(_.references))
-    val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
-
-    val candidatePredicates = filterPredicates.map {
-      _ transform {
-        case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes.
-      }
-    }
-
-    val (unhandledPredicates, pushedFilters) =
-      selectFilters(relation.relation, candidatePredicates)
-
-    // A set of column attributes that are only referenced by pushed down filters.  We can eliminate
-    // them from requested columns.
-    val handledSet = {
-      val handledPredicates = filterPredicates.filterNot(unhandledPredicates.contains)
-      val unhandledSet = AttributeSet(unhandledPredicates.flatMap(_.references))
-      AttributeSet(handledPredicates.flatMap(_.references)) --
-      (projectSet ++ unhandledSet).map(relation.attributeMap)
-    }
-
-    // Combines all Catalyst filter `Expression`s that are either not convertible to data source
-    // `Filter`s or cannot be handled by `relation`.
-    val filterCondition = unhandledPredicates.reduceLeftOption(expressions.And)
-    val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-    val map = table.carbonRelation.metaData.dictionaryMap
-
-    val metadata: Map[String, String] = {
-      val pairs = ArrayBuffer.empty[(String, String)]
-
-      if (pushedFilters.nonEmpty) {
-        pairs += (PUSHED_FILTERS -> pushedFilters.mkString("[", ", ", "]"))
-      }
-      pairs.toMap
-    }
-
-
-    val needDecoder = ArrayBuffer[AttributeReference]()
-    filterCondition match {
-      case Some(exp: Expression) =>
-        exp.references.collect {
-          case attr: AttributeReference =>
-            val dict = map.get(attr.name)
-            if (dict.isDefined && dict.get) {
-              needDecoder += attr
-            }
-        }
-      case None =>
-    }
-
-    projects.map {
-      case attr: AttributeReference =>
-      case Alias(attr: AttributeReference, _) =>
-      case others =>
-        others.references.map { f =>
-          val dictionary = map.get(f.name)
-          if (dictionary.isDefined && dictionary.get) {
-            needDecoder += f.asInstanceOf[AttributeReference]
-          }
-        }
-    }
-
-    if (projects.map(_.toAttribute) == projects &&
-        projectSet.size == projects.size &&
-        filterSet.subsetOf(projectSet)) {
-      // When it is possible to just use column pruning to get the right projection and
-      // when the columns of this projection are enough to evaluate all filter conditions,
-      // just do a scan followed by a filter, with no extra project.
-      val requestedColumns = projects
-        // Safe due to if above.
-        .asInstanceOf[Seq[Attribute]]
-        // Match original case of attributes.
-        .map(relation.attributeMap)
-        // Don't request columns that are only referenced by pushed filters.
-        .filterNot(handledSet.contains)
-      val updateRequestedColumns = updateRequestedColumnsFunc(requestedColumns, table, needDecoder)
-
-      val updateProject = projects.map { expr =>
-        var attr = expr.toAttribute.asInstanceOf[AttributeReference]
-        if (!needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
-          val dict = map.get(attr.name)
-          if (dict.isDefined && dict.get) {
-            attr = AttributeReference(attr.name, IntegerType, attr.nullable, attr.metadata)(attr
-              .exprId, attr.qualifier)
-          }
-        }
-        attr
-      }
-      val scan = getDataSourceScan(relation,
-        updateProject,
-        scanBuilder,
-        candidatePredicates,
-        pushedFilters,
-        metadata,
-        needDecoder,
-        updateRequestedColumns.asInstanceOf[Seq[Attribute]])
-      filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan)
-    } else {
-
-      var newProjectList: Seq[Attribute] = Seq.empty
-      val updatedProjects = projects.map {
-          case a@Alias(s: ScalaUDF, name)
-            if name.equalsIgnoreCase(CarbonCommonConstants.POSITION_ID) ||
-                name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID) =>
-            val reference = AttributeReference(name, StringType, true)().withExprId(a.exprId)
-            newProjectList :+= reference
-            reference
-          case a@Alias(s: ScalaUDF, name)
-            if name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID) =>
-            val reference =
-              AttributeReference(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID,
-                StringType, true)().withExprId(a.exprId)
-            newProjectList :+= reference
-            a.transform {
-              case s: ScalaUDF =>
-                ScalaUDF(s.function, s.dataType, Seq(reference), s.inputTypes)
-            }
-          case other => other
-      }
-      // Don't request columns that are only referenced by pushed filters.
-      val requestedColumns =
-        (projectSet ++ filterSet -- handledSet).map(relation.attributeMap).toSeq ++ newProjectList
-      val updateRequestedColumns = updateRequestedColumnsFunc(requestedColumns, table, needDecoder)
-      val scan = getDataSourceScan(relation,
-        updateRequestedColumns.asInstanceOf[Seq[Attribute]],
-        scanBuilder,
-        candidatePredicates,
-        pushedFilters,
-        metadata,
-        needDecoder,
-        updateRequestedColumns.asInstanceOf[Seq[Attribute]])
-      execution.ProjectExec(
-        updateRequestedColumnsFunc(updatedProjects, table,
-          needDecoder).asInstanceOf[Seq[NamedExpression]],
-        filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan))
-    }
-  }
-
-  def getDataSourceScan(relation: LogicalRelation,
-      output: Seq[Attribute],
-      scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
-        ArrayBuffer[AttributeReference]) => RDD[InternalRow],
-      candidatePredicates: Seq[Expression],
-      pushedFilters: Seq[Filter],
-      metadata: Map[String, String],
-      needDecoder: ArrayBuffer[AttributeReference],
-      updateRequestedColumns: Seq[Attribute]): DataSourceScanExec = {
-    val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-    if (supportBatchedDataSource(relation.relation.sqlContext, updateRequestedColumns) &&
-        needDecoder.isEmpty) {
-      BatchedDataSourceScanExec(
-        output,
-        scanBuilder(updateRequestedColumns, candidatePredicates, pushedFilters, needDecoder),
-        relation.relation,
-        getPartitioning(table.carbonTable, updateRequestedColumns),
-        metadata,
-        relation.catalogTable.map(_.identifier))
-    } else {
-      RowDataSourceScanExec(output,
-        scanBuilder(updateRequestedColumns, candidatePredicates, pushedFilters, needDecoder),
-        relation.relation,
-        getPartitioning(table.carbonTable, updateRequestedColumns),
-        metadata,
-        relation.catalogTable.map(_.identifier))
-    }
-  }
-
-  def updateRequestedColumnsFunc(requestedColumns: Seq[Expression],
-      relation: CarbonDatasourceHadoopRelation,
-      needDecoder: ArrayBuffer[AttributeReference]): Seq[Expression] = {
-    val map = relation.carbonRelation.metaData.dictionaryMap
-    requestedColumns.map {
-      case attr: AttributeReference =>
-        if (needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
-          attr
-        } else {
-          val dict = map.get(attr.name)
-          if (dict.isDefined && dict.get) {
-            AttributeReference(attr.name,
-              IntegerType,
-              attr.nullable,
-              attr.metadata)(attr.exprId, attr.qualifier)
-          } else {
-            attr
-          }
-        }
-      case alias @ Alias(attr: AttributeReference, name) =>
-        if (needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
-          alias
-        } else {
-          val dict = map.get(attr.name)
-          if (dict.isDefined && dict.get) {
-            alias.transform {
-              case attrLocal: AttributeReference =>
-                AttributeReference(attr.name,
-                  IntegerType,
-                  attr.nullable,
-                  attr.metadata)(attr.exprId, attr.qualifier)
-            }
-          } else {
-            alias
-          }
-        }
-      case others => others
-    }
-  }
-
-  private def getPartitioning(carbonTable: CarbonTable,
-      output: Seq[Attribute]): Partitioning = {
-    val info: BucketingInfo = carbonTable.getBucketingInfo(carbonTable.getFactTableName)
-    if (info != null) {
-      val cols = info.getListOfColumns.asScala
-      val sortColumn = carbonTable.
-        getDimensionByTableName(carbonTable.getFactTableName).get(0).getColName
-      val numBuckets = info.getNumberOfBuckets
-      val bucketColumns = cols.flatMap { n =>
-        val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
-        attrRef match {
-          case Some(attr: AttributeReference) =>
-            Some(AttributeReference(attr.name,
-              CarbonScalaUtil.convertCarbonToSparkDataType(n.getDataType),
-              attr.nullable,
-              attr.metadata)(attr.exprId, attr.qualifier))
-          case _ => None
-        }
-      }
-      if (bucketColumns.size == cols.size) {
-        HashPartitioning(bucketColumns, numBuckets)
-      } else {
-        UnknownPartitioning(0)
-      }
-    } else {
-      UnknownPartitioning(0)
-    }
-  }
-
-  protected[sql] def selectFilters(
-      relation: BaseRelation,
-      predicates: Seq[Expression]): (Seq[Expression], Seq[Filter]) = {
-
-    // For conciseness, all Catalyst filter expressions of type `expressions.Expression` below are
-    // called `predicate`s, while all data source filters of type `sources.Filter` are simply called
-    // `filter`s.
-
-    val translated: Seq[(Expression, Filter)] =
-      for {
-        predicate <- predicates
-        filter <- translateFilter(predicate)
-      } yield predicate -> filter
-
-    // A map from original Catalyst expressions to corresponding translated data source filters.
-    val translatedMap: Map[Expression, Filter] = translated.toMap
-
-    // Catalyst predicate expressions that cannot be translated to data source filters.
-    val unrecognizedPredicates = predicates.filterNot(translatedMap.contains)
-
-    // Data source filters that cannot be handled by `relation`. The semantic of a unhandled filter
-    // at here is that a data source may not be able to apply this filter to every row
-    // of the underlying dataset.
-    val unhandledFilters = relation.unhandledFilters(translatedMap.values.toArray).toSet
-
-    val (unhandled, handled) = translated.partition {
-      case (predicate, filter) =>
-        unhandledFilters.contains(filter)
-    }
-
-    // Catalyst predicate expressions that can be translated to data source filters, but cannot be
-    // handled by `relation`.
-    val (unhandledPredicates, _) = unhandled.unzip
-
-    // Translated data source filters that can be handled by `relation`
-    val (_, handledFilters) = handled.unzip
-
-    // translated contains all filters that have been converted to the public Filter interface.
-    // We should always push them to the data source no matter whether the data source can apply
-    // a filter to every row or not.
-    val (_, translatedFilters) = translated.unzip
-
-    (unrecognizedPredicates ++ unhandledPredicates, translatedFilters)
-  }
-
-
-  /**
-   * Tries to translate a Catalyst [[Expression]] into data source [[Filter]].
-   *
-   * @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`.
-   */
-  protected[sql] def translateFilter(predicate: Expression, or: Boolean = false): Option[Filter] = {
-    predicate match {
-      case or@Or(left, right) =>
-
-        val leftFilter = translateFilter(left, true)
-        val rightFilter = translateFilter(right, true)
-        if (leftFilter.isDefined && rightFilter.isDefined) {
-          Some(sources.Or(leftFilter.get, rightFilter.get))
-        } else {
-          None
-        }
-
-      case And(left, right) =>
-        val leftFilter = translateFilter(left, or)
-        val rightFilter = translateFilter(right, or)
-        if (or) {
-          if (leftFilter.isDefined && rightFilter.isDefined) {
-            (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
-          } else {
-            None
-          }
-        } else {
-          (translateFilter(left) ++ translateFilter(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 c@EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case Not(EqualTo(a: Attribute, Literal(v, t))) =>
-        Some(sources.Not(sources.EqualTo(a.name, v)))
-      case Not(EqualTo(Literal(v, t), a: Attribute)) =>
-        Some(sources.Not(sources.EqualTo(a.name, v)))
-      case c@Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      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 c@Not(In(Cast(a: Attribute, _), list))
-        if !list.exists(!_.isInstanceOf[Literal]) =>
-        Some(CastExpr(c))
-      case c@In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
-        Some(CastExpr(c))
-      case InSet(a: Attribute, set) =>
-        Some(sources.In(a.name, set.toArray))
-      case Not(InSet(a: Attribute, set)) =>
-        Some(sources.Not(sources.In(a.name, set.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 c@GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      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 c@LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      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 c@GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      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 c@LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case StartsWith(a: Attribute, Literal(v, t)) =>
-        Some(sources.StringStartsWith(a.name, v.toString))
-      case c@EndsWith(a: Attribute, Literal(v, t)) =>
-        Some(CarbonEndsWith(c))
-      case c@Contains(a: Attribute, Literal(v, t)) =>
-        Some(CarbonContainsWith(c))
-      case others => None
-    }
-  }
-
-  def supportBatchedDataSource(sqlContext: SQLContext, cols: Seq[Attribute]): Boolean = {
-    val vectorizedReader = {
-      if (sqlContext.sparkSession.conf.contains(CarbonCommonConstants.ENABLE_VECTOR_READER)) {
-        sqlContext.sparkSession.conf.get(CarbonCommonConstants.ENABLE_VECTOR_READER)
-      } else if (System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER) != null) {
-        System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER)
-      } else {
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER,
-          CarbonCommonConstants.ENABLE_VECTOR_READER_DEFAULT)
-      }
-    }
-    val supportCodegen =
-      sqlContext.conf.wholeStageEnabled && sqlContext.conf.wholeStageMaxNumFields >= cols.size
-    supportCodegen && vectorizedReader.toBoolean &&
-    cols.forall(_.dataType.isInstanceOf[AtomicType])
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
deleted file mode 100644
index 17e456d..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
+++ /dev/null
@@ -1,460 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-import scala.language.implicitConversions
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
-import org.apache.spark.util.AlterTableUtil
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
-
-private[sql] case class AlterTableAddColumns(
-    alterTableAddColumnsModel: AlterTableAddColumnsModel) extends RunnableCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableAddColumnsModel.tableName
-    val dbName = alterTableAddColumnsModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    var newCols = Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]()
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      // Consider a concurrent scenario where 2 alter operations are executed in parallel. 1st
-      // operation is success and updates the schema file. 2nd operation will get the lock after
-      // completion of 1st operation but as look up relation is called before it will have the
-      // older carbon table and this can lead to inconsistent state in the system. Therefor look
-      // up relation should be called after acquiring the lock
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.carbonTable
-      // get the latest carbon table and check for column existence
-      // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-      val wrapperTableInfo = schemaConverter
-        .fromExternalToWrapperTableInfo(thriftTableInfo,
-          dbName,
-          tableName,
-          carbonTable.getStorePath)
-      newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
-        dbName,
-        wrapperTableInfo,
-        carbonTablePath,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath, sparkSession.sparkContext).process
-      // generate dictionary files for the newly added columns
-      new AlterTableAddColumnRDD(sparkSession.sparkContext,
-        newCols,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath).collect()
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
-      schemaEvolutionEntry.setTimeStamp(timeStamp)
-      schemaEvolutionEntry.setAdded(newCols.toList.asJava)
-      val thriftTable = schemaConverter
-        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
-          thriftTable)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      LOGGER.info(s"Alter table for add columns is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for add columns is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception =>
-        LOGGER.error(e, "Alter table add columns failed")
-        if (newCols.nonEmpty) {
-          LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
-          new AlterTableDropColumnRDD(sparkSession.sparkContext,
-            newCols,
-            carbonTable.getCarbonTableIdentifier,
-            carbonTable.getStorePath).collect()
-          AlterTableUtil.revertAddColumnChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table add operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}
-
-private[sql] case class AlterTableRenameTable(alterTableRenameModel: AlterTableRenameModel)
-  extends RunnableCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    val oldTableIdentifier = alterTableRenameModel.oldTableIdentifier
-    val newTableIdentifier = alterTableRenameModel.newTableIdentifier
-    val oldDatabaseName = oldTableIdentifier.database
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    val newDatabaseName = newTableIdentifier.database
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    if (!oldDatabaseName.equalsIgnoreCase(newDatabaseName)) {
-      throw new MalformedCarbonCommandException("Database name should be same for both tables")
-    }
-    val tableExists = sparkSession.catalog.tableExists(oldDatabaseName, newTableIdentifier.table)
-    if (tableExists) {
-      throw new MalformedCarbonCommandException(s"Table with name $newTableIdentifier " +
-                                                s"already exists")
-    }
-    val oldTableName = oldTableIdentifier.table.toLowerCase
-    val newTableName = newTableIdentifier.table.toLowerCase
-    LOGGER.audit(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
-    LOGGER.info(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
-    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    val relation: CarbonRelation =
-      metastore.lookupRelation(oldTableIdentifier.database, oldTableName)(sparkSession)
-        .asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      LOGGER.audit(s"Rename table request has failed. " +
-                   s"Table $oldDatabaseName.$oldTableName does not exist")
-      sys.error(s"Table $oldDatabaseName.$oldTableName does not exist")
-    }
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-      LockUsage.COMPACTION_LOCK,
-      LockUsage.DELETE_SEGMENT_LOCK,
-      LockUsage.CLEAN_FILES_LOCK,
-      LockUsage.DROP_TABLE_LOCK)
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(oldDatabaseName, oldTableName, locksToBeAcquired)(
-            sparkSession)
-      val tableMeta = metastore.lookupRelation(Some(oldDatabaseName), oldTableName)(sparkSession)
-        .asInstanceOf[CarbonRelation].tableMeta
-      carbonTable = tableMeta.carbonTable
-      // get the latest carbon table and check for column existence
-      val carbonTablePath = CarbonStorePath.
-        getCarbonTablePath(AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath))
-      val tableMetadataFile = carbonTablePath.getPath
-      val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
-      schemaEvolutionEntry.setTableName(newTableName)
-      timeStamp = System.currentTimeMillis()
-      schemaEvolutionEntry.setTime_stamp(timeStamp)
-      renameBadRecords(oldTableName, newTableName, oldDatabaseName)
-      val fileType = FileFactory.getFileType(tableMetadataFile)
-      if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
-        val rename = FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
-          .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
-                       newTableName)
-        if (!rename) {
-          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
-          sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
-        }
-      }
-      val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
-        newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
-      val newTablePath = metastore.updateTableSchema(newTableIdentifier,
-          carbonTable.getCarbonTableIdentifier,
-          tableInfo,
-          schemaEvolutionEntry,
-          tableMeta.tablePath)(sparkSession)
-      metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
-      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
-        .runSqlHive(
-          s"ALTER TABLE $oldDatabaseName.$oldTableName RENAME TO $oldDatabaseName.$newTableName")
-      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
-        .runSqlHive(
-          s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
-          s"('tableName'='$newTableName', " +
-          s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
-      sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
-        Some(oldDatabaseName)).quotedString)
-      LOGGER.audit(s"Table $oldTableName has been successfully renamed to $newTableName")
-      LOGGER.info(s"Table $oldTableName has been successfully renamed to $newTableName")
-    } catch {
-      case e: Exception =>
-        LOGGER.error(e, "Rename table failed: " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil
-            .revertRenameTableChanges(oldTableIdentifier,
-              newTableName,
-              carbonTable.getStorePath,
-              carbonTable.getCarbonTableIdentifier.getTableId,
-              timeStamp)(
-              sparkSession)
-          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
-        }
-        sys.error(s"Alter table rename table operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-      // case specific to rename table as after table rename old table path will not be found
-      if (carbonTable != null) {
-        AlterTableUtil
-          .releaseLocksManually(locks,
-            locksToBeAcquired,
-            oldDatabaseName,
-            newTableName,
-            carbonTable.getStorePath)
-      }
-    }
-    Seq.empty
-  }
-
-  private def renameBadRecords(oldTableName: String,
-      newTableName: String,
-      dataBaseName: String) = {
-    val oldPath = CarbonUtil
-      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + oldTableName)
-    val newPath = CarbonUtil
-      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + newTableName)
-    val fileType = FileFactory.getFileType(oldPath)
-    if (FileFactory.isFileExist(oldPath, fileType)) {
-      val renameSuccess = FileFactory.getCarbonFile(oldPath, fileType)
-        .renameForce(newPath)
-      if (!renameSuccess) {
-        sys.error(s"BadRecords Folder Rename Failed for table $dataBaseName.$oldTableName")
-      }
-    }
-  }
-
-}
-
-private[sql] case class AlterTableDropColumns(
-    alterTableDropColumnModel: AlterTableDropColumnModel) extends RunnableCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableDropColumnModel.tableName
-    val dbName = alterTableDropColumnModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table drop columns request has been received for $dbName.$tableName")
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    // get the latest carbon table and check for column existence
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.carbonTable
-      val partitionInfo = carbonTable.getPartitionInfo(tableName)
-      if (partitionInfo != null) {
-        val partitionColumnSchemaList = partitionInfo.getColumnSchemaList.asScala
-          .map(_.getColumnName)
-        // check each column existence in the table
-        val partitionColumns = alterTableDropColumnModel.columns.filter {
-          tableColumn => partitionColumnSchemaList.contains(tableColumn)
-        }
-        if (partitionColumns.nonEmpty) {
-          throw new UnsupportedOperationException("Partition columns cannot be dropped: " +
-                                                  s"$partitionColumns")
-        }
-      }
-      val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
-      var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column
-      .ColumnSchema]()
-      var keyColumnCountToBeDeleted = 0
-      // TODO: if deleted column list includes bucketted column throw an error
-      alterTableDropColumnModel.columns.foreach { column =>
-        var columnExist = false
-        tableColumns.foreach { tableColumn =>
-          // column should not be already deleted and should exist in the table
-          if (!tableColumn.isInvisible && column.equalsIgnoreCase(tableColumn.getColName)) {
-            if (tableColumn.isDimension) {
-              keyColumnCountToBeDeleted += 1
-              if (tableColumn.hasEncoding(Encoding.DICTIONARY)) {
-                dictionaryColumns ++= Seq(tableColumn.getColumnSchema)
-              }
-            }
-            columnExist = true
-          }
-        }
-        if (!columnExist) {
-          sys.error(s"Column $column does not exists in the table $dbName.$tableName")
-        }
-      }
-      // take the total key column count. key column to be deleted should not
-      // be >= key columns in schema
-      val totalKeyColumnInSchema = tableColumns.count {
-        tableColumn => !tableColumn.isInvisible && tableColumn.isDimension
-      }
-      if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
-        sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
-      }
-      // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
-      val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      // maintain the deleted columns for schema evolution history
-      var deletedColumnSchema = ListBuffer[org.apache.carbondata.format.ColumnSchema]()
-      val columnSchemaList = tableInfo.fact_table.table_columns.asScala
-      alterTableDropColumnModel.columns.foreach { column =>
-        columnSchemaList.foreach { columnSchema =>
-          if (!columnSchema.invisible && column.equalsIgnoreCase(columnSchema.column_name)) {
-            deletedColumnSchema += columnSchema.deepCopy
-            columnSchema.invisible = true
-          }
-        }
-      }
-      // add deleted columns to schema evolution history and update the schema
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
-      schemaEvolutionEntry.setRemoved(deletedColumnSchema.toList.asJava)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaEvolutionEntry,
-          tableInfo)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      // TODO: 1. add check for deletion of index tables
-      // delete dictionary files for dictionary column and clear dictionary cache from memory
-      new AlterTableDropColumnRDD(sparkSession.sparkContext,
-        dictionaryColumns,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath).collect()
-      LOGGER.info(s"Alter table for drop columns is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for drop columns is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception => LOGGER
-        .error("Alter table drop columns failed : " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil.revertDropColumnChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table drop column operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}
-
-private[sql] case class AlterTableDataTypeChange(
-    alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel) extends RunnableCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableDataTypeChangeModel.tableName
-    val dbName = alterTableDataTypeChangeModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    var locks = List.empty[ICarbonLock]
-    // get the latest carbon table and check for column existence
-    var carbonTable: CarbonTable = null
-    var timeStamp = 0L
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.carbonTable
-      val columnName = alterTableDataTypeChangeModel.columnName
-      val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala.filter(!_.isInvisible)
-      if (!carbonColumns.exists(_.getColName.equalsIgnoreCase(columnName))) {
-        LOGGER.audit(s"Alter table change data type request has failed. " +
-                     s"Column $columnName does not exist")
-        sys.error(s"Column does not exist: $columnName")
-      }
-      val carbonColumn = carbonColumns.filter(_.getColName.equalsIgnoreCase(columnName))
-      if (carbonColumn.size == 1) {
-        CarbonScalaUtil
-          .validateColumnDataType(alterTableDataTypeChangeModel.dataTypeInfo, carbonColumn(0))
-      } else {
-        LOGGER.audit(s"Alter table change data type request has failed. " +
-                     s"Column $columnName is invalid")
-        sys.error(s"Invalid Column: $columnName")
-      }
-      // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
-      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      // maintain the added column for schema evolution history
-      var addColumnSchema: ColumnSchema = null
-      var deletedColumnSchema: ColumnSchema = null
-      val columnSchemaList = tableInfo.fact_table.table_columns.asScala.filter(!_.isInvisible)
-      columnSchemaList.foreach { columnSchema =>
-        if (columnSchema.column_name.equalsIgnoreCase(columnName)) {
-          deletedColumnSchema = columnSchema.deepCopy
-          columnSchema.setData_type(DataTypeConverterUtil
-            .convertToThriftDataType(alterTableDataTypeChangeModel.dataTypeInfo.dataType))
-          columnSchema.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
-          columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
-          addColumnSchema = columnSchema
-        }
-      }
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
-      schemaEvolutionEntry.setAdded(List(addColumnSchema).asJava)
-      schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava)
-      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-        .setTime_stamp(System.currentTimeMillis)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaEvolutionEntry,
-          tableInfo)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      LOGGER.info(s"Alter table for data type change is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for data type change is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception => LOGGER
-        .error("Alter table change datatype failed : " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil.revertDataTypeChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table data type change operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
new file mode 100644
index 0000000..f5c6cba
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
@@ -0,0 +1,107 @@
+/*
+ * 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 org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.exception.InvalidConfigurationException
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.schema.table.TableInfo
+import org.apache.carbondata.core.util.CarbonUtil
+
+case class CarbonCreateTableCommand(
+    cm: TableModel,
+    createDSTable: Boolean = true)
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val storePath = CarbonEnv.getInstance(sparkSession).storePath
+    CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      checkSchemasModifiedTimeAndReloadTables(storePath)
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    cm.databaseName = GetDB.getDatabaseName(cm.databaseNameOp, sparkSession)
+    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)
+
+    // Add validation for sort scope when create table
+    val sortScope = tableInfo.getFactTable.getTableProperties
+      .getOrDefault("sort_scope", CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT)
+    if (!CarbonUtil.isValidSortOption(sortScope)) {
+      throw new InvalidConfigurationException(
+        s"Passing invalid SORT_SCOPE '$sortScope', valid SORT_SCOPE are 'NO_SORT', 'BATCH_SORT'," +
+        s" 'LOCAL_SORT' and 'GLOBAL_SORT' ")
+    }
+
+    if (tableInfo.getFactTable.getListOfColumns.size <= 0) {
+      sys.error("No Dimensions found. Table should have at least one dimesnion !")
+    }
+
+    if (sparkSession.sessionState.catalog.listTables(dbName)
+      .exists(_.table.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 {
+      val tableIdentifier = AbsoluteTableIdentifier.from(storePath, dbName, tbName)
+      // Add Database to catalog and persist
+      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val tablePath = tableIdentifier.getTablePath
+      val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tablePath)
+      if (createDSTable) {
+        try {
+          val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
+          cm.dimCols.foreach(f => fields(f.schemaOrdinal) = f)
+          cm.msrCols.foreach(f => fields(f.schemaOrdinal) = f)
+
+          sparkSession.sql(
+            s"""CREATE TABLE $dbName.$tbName
+               |(${ fields.map(f => f.rawSchema).mkString(",") })
+               |USING org.apache.spark.sql.CarbonSource""".stripMargin +
+            s""" OPTIONS (tableName "$tbName", dbName "$dbName", tablePath """.stripMargin +
+            s""""$tablePath"$carbonSchemaString) """)
+        } catch {
+          case e: Exception =>
+            val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
+            // call the drop table to delete the created table.
+            CarbonEnv.getInstance(sparkSession).carbonMetastore
+              .dropTable(tablePath, 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
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
new file mode 100644
index 0000000..e5f6b75
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
@@ -0,0 +1,133 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.hive.CarbonRelation
+import org.codehaus.jackson.map.ObjectMapper
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
+
+private[sql] case class CarbonDescribeFormattedCommand(
+    child: SparkPlan,
+    override val output: Seq[Attribute],
+    tblIdentifier: TableIdentifier)
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  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.foreach { x =>
+      results = results :+ (s"Column Group $index", x, "")
+      index = index + 1
+    }
+    results
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(tblIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
+    val mapper = new ObjectMapper()
+    val colProps = StringBuilder.newBuilder
+    val dims = relation.metaData.dims.map(x => x.toLowerCase)
+    var results: Seq[(String, String, String)] = child.schema.fields.map { field =>
+      val fieldName = field.name.toLowerCase
+      val comment = if (dims.contains(fieldName)) {
+        val dimension = relation.metaData.carbonTable.getDimensionByName(
+          relation.tableMeta.carbonTableIdentifier.getTableName, fieldName)
+        if (null != dimension.getColumnProperties && !dimension.getColumnProperties.isEmpty) {
+          colProps.append(fieldName).append(".")
+            .append(mapper.writeValueAsString(dimension.getColumnProperties))
+            .append(",")
+        }
+        if (dimension.hasEncoding(Encoding.DICTIONARY) &&
+            !dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          "DICTIONARY, KEY COLUMN" + (if (dimension.hasEncoding(Encoding.INVERTED_INDEX)) {
+            ""
+          } else {
+            ",NOINVERTEDINDEX"
+          })
+        } else {
+          "KEY COLUMN" + (if (dimension.hasEncoding(Encoding.INVERTED_INDEX)) {
+            ""
+          } else {
+            ",NOINVERTEDINDEX"
+          })
+        }
+      } 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(("SORT_SCOPE", carbonTable.getTableInfo.getFactTable
+      .getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants
+      .LOAD_SORT_SCOPE_DEFAULT), CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
+    results ++= Seq(("", "", ""), ("##Detailed Column property", "", ""))
+    if (colPropStr.length() > 0) {
+      results ++= Seq((colPropStr, "", ""))
+    } else {
+      results ++= Seq(("ADAPTIVE", "", ""))
+    }
+    results ++= Seq(("SORT_COLUMNS", relation.metaData.carbonTable.getSortColumns(
+      relation.tableMeta.carbonTableIdentifier.getTableName).asScala
+      .map(column => column).mkString(","), ""))
+    val dimension = carbonTable
+      .getDimensionByTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
+    results ++= getColumnGroups(dimension.asScala.toList)
+    if (carbonTable.getPartitionInfo(carbonTable.getFactTableName) != null) {
+      results ++=
+      Seq(("Partition Columns: ", carbonTable.getPartitionInfo(carbonTable.getFactTableName)
+        .getColumnSchemaList.asScala.map(_.getColumnName).mkString(","), ""))
+    }
+    results.map { case (name, dataType, comment) =>
+      Row(f"$name%-36s", f"$dataType%-80s", f"$comment%-72s")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
new file mode 100644
index 0000000..5f70771
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
@@ -0,0 +1,94 @@
+/*
+ * 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 scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonStorePath
+
+case class CarbonDropTableCommand(
+    ifExistsSet: Boolean,
+    databaseNameOp: Option[String],
+    tableName: String)
+  extends RunnableCommand with SchemaProcessCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+    processData(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val dbName = GetDB.getDatabaseName(databaseNameOp, sparkSession)
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName, "")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
+    val carbonEnv = CarbonEnv.getInstance(sparkSession)
+    val catalog = carbonEnv.carbonMetastore
+    val tableIdentifier =
+      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath,
+        dbName.toLowerCase, tableName.toLowerCase)
+    catalog.checkSchemasModifiedTimeAndReloadTables(tableIdentifier.getStorePath)
+    val carbonLocks: scala.collection.mutable.ListBuffer[ICarbonLock] = ListBuffer()
+    try {
+      locksToBeAcquired foreach {
+        lock => carbonLocks += CarbonLockUtil.getLockObject(carbonTableIdentifier, lock)
+      }
+      LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
+
+      CarbonEnv.getInstance(sparkSession).carbonMetastore
+        .dropTable(tableIdentifier.getTablePath, identifier)(sparkSession)
+      LOGGER.audit(s"Deleted table [$tableName] under database [$dbName]")
+    } catch {
+      case ex: Exception =>
+        LOGGER.error(ex, s"Dropping table $dbName.$tableName failed")
+        sys.error(s"Dropping table $dbName.$tableName failed: ${ex.getMessage}")
+    } finally {
+      if (carbonLocks.nonEmpty) {
+        val unlocked = carbonLocks.forall(_.unlock())
+        if (unlocked) {
+          logInfo("Table MetaData Unlocked Successfully")
+        }
+      }
+    }
+    Seq.empty
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    // delete the table folder
+    val dbName = GetDB.getDatabaseName(databaseNameOp, sparkSession)
+    val tableIdentifier =
+      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath, dbName, tableName)
+    val metadataFilePath =
+      CarbonStorePath.getCarbonTablePath(tableIdentifier).getMetadataDirectoryPath
+    val fileType = FileFactory.getFileType(metadataFilePath)
+    if (FileFactory.isFileExist(metadataFilePath, fileType)) {
+      val file = FileFactory.getCarbonFile(metadataFilePath, fileType)
+      CarbonUtil.deleteFoldersAndFilesSilent(file.getParentFile)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
deleted file mode 100644
index c8c716e..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql.execution.command
-
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
-import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
-
-import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-
-/**
- * Carbon strategies for ddl commands
- */
-class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
-
-  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
-    plan match {
-      case LoadDataCommand(identifier, path, isLocal, isOverwrite, partition)
-        if CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(identifier)(sparkSession) =>
-        ExecutedCommandExec(LoadTable(identifier.database, identifier.table.toLowerCase, path,
-          Seq(), Map(), isOverwrite)) :: Nil
-      case alter@AlterTableRenameCommand(oldTableIdentifier, newTableIdentifier, _) =>
-        val dbOption = oldTableIdentifier.database.map(_.toLowerCase)
-        val tableIdentifier = TableIdentifier(oldTableIdentifier.table.toLowerCase(), dbOption)
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(tableIdentifier)(
-            sparkSession)
-        if (isCarbonTable) {
-          val renameModel = AlterTableRenameModel(tableIdentifier, newTableIdentifier)
-          ExecutedCommandExec(AlterTableRenameTable(renameModel)) :: Nil
-        } else {
-          ExecutedCommandExec(alter) :: Nil
-        }
-      case DropTableCommand(identifier, ifNotExists, isView, _)
-        if CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .isTablePathExists(identifier)(sparkSession) =>
-        ExecutedCommandExec(
-          CarbonDropTableCommand(ifNotExists, identifier.database,
-            identifier.table.toLowerCase)) :: Nil
-      case ShowLoadsCommand(databaseName, table, limit) =>
-        ExecutedCommandExec(ShowLoads(databaseName, table.toLowerCase, limit, plan.output)) :: Nil
-      case InsertIntoCarbonTable(relation: CarbonDatasourceHadoopRelation,
-      _, child: LogicalPlan, overwrite, _) =>
-        ExecutedCommandExec(LoadTableByInsert(relation, child, overwrite.enabled)) :: Nil
-      case createDb@CreateDatabaseCommand(dbName, ifNotExists, _, _, _) =>
-        CarbonUtil.createDatabaseDirectory(dbName, CarbonEnv.getInstance(sparkSession).storePath)
-        ExecutedCommandExec(createDb) :: Nil
-      case drop@DropDatabaseCommand(dbName, ifExists, isCascade) =>
-        ExecutedCommandExec(CarbonDropDatabaseCommand(drop)) :: Nil
-      case alterTable@AlterTableCompaction(altertablemodel) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(TableIdentifier(altertablemodel.tableName,
-            altertablemodel.dbName))(sparkSession)
-        if (isCarbonTable) {
-          if (altertablemodel.compactionType.equalsIgnoreCase("minor") ||
-              altertablemodel.compactionType.equalsIgnoreCase("major")) {
-            ExecutedCommandExec(alterTable) :: Nil
-          } else {
-            throw new MalformedCarbonCommandException(
-              "Unsupported alter operation on carbon table")
-          }
-        } else {
-          throw new MalformedCarbonCommandException(
-            "Operation not allowed : " + altertablemodel.alterSql)
-        }
-      case dataTypeChange@AlterTableDataTypeChange(alterTableChangeDataTypeModel) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(TableIdentifier(alterTableChangeDataTypeModel.tableName,
-            alterTableChangeDataTypeModel.databaseName))(sparkSession)
-        if (isCarbonTable) {
-          ExecutedCommandExec(dataTypeChange) :: Nil
-        } else {
-          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
-        }
-      case addColumn@AlterTableAddColumns(alterTableAddColumnsModel) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(TableIdentifier(alterTableAddColumnsModel.tableName,
-            alterTableAddColumnsModel.databaseName))(sparkSession)
-        if (isCarbonTable) {
-          ExecutedCommandExec(addColumn) :: Nil
-        } else {
-          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
-        }
-      case dropColumn@AlterTableDropColumns(alterTableDropColumnModel) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(TableIdentifier(alterTableDropColumnModel.tableName,
-            alterTableDropColumnModel.databaseName))(sparkSession)
-        if (isCarbonTable) {
-          ExecutedCommandExec(dropColumn) :: Nil
-        } else {
-          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
-        }
-      case desc@DescribeTableCommand(identifier, partitionSpec, isExtended, isFormatted)
-        if CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(identifier)(sparkSession) && isFormatted =>
-        val resolvedTable =
-          sparkSession.sessionState.executePlan(UnresolvedRelation(identifier, None)).analyzed
-        val resultPlan = sparkSession.sessionState.executePlan(resolvedTable).executedPlan
-        ExecutedCommandExec(DescribeCommandFormatted(resultPlan, plan.output, identifier)) :: Nil
-      case ShowPartitionsCommand(t, cols) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(t)(sparkSession)
-        if (isCarbonTable) {
-          ExecutedCommandExec(ShowCarbonPartitionsCommand(t)) :: Nil
-        } else {
-          ExecutedCommandExec(ShowPartitionsCommand(t, cols)) :: Nil
-        }
-      case set@SetCommand(kv) =>
-        ExecutedCommandExec(CarbonSetCommand(set)) :: Nil
-      case reset@ResetCommand =>
-        ExecutedCommandExec(CarbonResetCommand()) :: Nil
-      case org.apache.spark.sql.execution.datasources.CreateTable(tableDesc, mode, None)
-        if tableDesc.provider.get != DDLUtils.HIVE_PROVIDER
-           && tableDesc.provider.get.equals("org.apache.spark.sql.CarbonSource") =>
-        val updatedCatalog =
-          CarbonSource.updateCatalogTableWithCarbonSchema(tableDesc, sparkSession)
-        val cmd =
-          CreateDataSourceTableCommand(updatedCatalog, ignoreIfExists = mode == SaveMode.Ignore)
-        ExecutedCommandExec(cmd) :: Nil
-      case _ => Nil
-    }
-  }
-
-}


[2/6] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by qi...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
new file mode 100644
index 0000000..21b974a
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
@@ -0,0 +1,187 @@
+/*
+ * 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.partition
+
+import java.text.SimpleDateFormat
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableSplitPartitionModel, DataProcessCommand, RunnableCommand, SchemaProcessCommand}
+import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
+import org.apache.spark.util.{AlterTableUtil, PartitionUtils}
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.cache.CacheProvider
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.PartitionInfo
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+
+/**
+ * Command for Alter Table Add & Split partition
+ * Add is a special case of Splitting the default partition (part0)
+ */
+case class AlterTableSplitCarbonPartitionCommand(
+    splitPartitionModel: AlterTableSplitPartitionModel)
+  extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
+
+  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
+
+  // TODO will add rollback function in case of process data failure
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+    processData(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+    val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
+    val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val tableName = splitPartitionModel.tableName
+    val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
+      .asInstanceOf[CarbonRelation]
+    val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+    val storePath = relation.tableMeta.storePath
+    if (relation == null) {
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
+    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
+      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
+      sys.error(s"Alter table failed. table not found: $dbName.$tableName")
+    }
+    val table = relation.tableMeta.carbonTable
+    val partitionInfo = table.getPartitionInfo(tableName)
+    val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
+    // keep a copy of partitionIdList before update partitionInfo.
+    // will be used in partition data scan
+    oldPartitionIds.addAll(partitionIds.asJava)
+
+    if (partitionInfo == null) {
+      sys.error(s"Table $tableName is not a partition table.")
+    }
+    if (partitionInfo.getPartitionType == PartitionType.HASH) {
+      sys.error(s"Hash partition table cannot be added or split!")
+    }
+
+    updatePartitionInfo(partitionInfo, partitionIds)
+
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
+    val schemaFilePath = carbonTablePath.getSchemaFilePath
+    // read TableInfo
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
+    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
+      dbName, tableName, storePath)
+    val tableSchema = wrapperTableInfo.getFactTable
+    tableSchema.setPartitionInfo(partitionInfo)
+    wrapperTableInfo.setFactTable(tableSchema)
+    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
+    val thriftTable =
+      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+    carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
+      dbName, tableName, storePath)
+    CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
+    // update the schema modified time
+    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
+    sparkSession.catalog.refreshTable(tableName)
+    Seq.empty
+  }
+
+  private def updatePartitionInfo(partitionInfo: PartitionInfo,
+      partitionIds: List[Int]) = {
+    val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
+      .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+
+    val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
+      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+
+    PartitionUtils.updatePartitionInfo(
+      partitionInfo,
+      partitionIds,
+      splitPartitionModel.partitionId.toInt,
+      splitPartitionModel.splitInfo,
+      timestampFormatter,
+      dateFormatter)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
+    val tableName = splitPartitionModel.tableName
+    var locks = List.empty[ICarbonLock]
+    var success = false
+    try {
+      val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+        LockUsage.COMPACTION_LOCK,
+        LockUsage.DELETE_SEGMENT_LOCK,
+        LockUsage.DROP_TABLE_LOCK,
+        LockUsage.CLEAN_FILES_LOCK,
+        LockUsage.ALTER_PARTITION_LOCK)
+      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
+        locksToBeAcquired)(sparkSession)
+      val carbonLoadModel = new CarbonLoadModel()
+      val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+      val storePath = relation.tableMeta.storePath
+      val table = relation.tableMeta.carbonTable
+      val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+      val dataLoadSchema = new CarbonDataLoadSchema(table)
+      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+      carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
+      carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
+      carbonLoadModel.setStorePath(storePath)
+      val loadStartTime = CarbonUpdateUtil.readCurrentTime
+      carbonLoadModel.setFactTimeStamp(loadStartTime)
+      CarbonDataRDDFactory.alterTableSplitPartition(
+        sparkSession.sqlContext,
+        splitPartitionModel.partitionId.toInt.toString,
+        carbonLoadModel,
+        oldPartitionIds.asScala.toList
+      )
+      success = true
+    } catch {
+      case e: Exception =>
+        success = false
+        sys.error(s"Add/Split Partition failed. Please check logs for more info. ${ e.getMessage }")
+    } finally {
+      AlterTableUtil.releaseLocks(locks)
+      CacheProvider.getInstance().dropAllCache()
+      val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+      LOGGER.info("Locks released after alter table add/split partition action.")
+      LOGGER.audit("Locks released after alter table add/split partition action.")
+      if (success) {
+        LOGGER.info(s"Alter table add/split partition is successful for table $dbName.$tableName")
+        LOGGER.audit(s"Alter table add/split partition is successful for table $dbName.$tableName")
+      }
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/ShowCarbonPartitionsCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/ShowCarbonPartitionsCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/ShowCarbonPartitionsCommand.scala
new file mode 100644
index 0000000..224304a
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/ShowCarbonPartitionsCommand.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.spark.sql.execution.command.partition
+
+import org.apache.spark.sql.{AnalysisException, CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.command.{RunnableCommand, SchemaProcessCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.spark.util.CommonUtil
+
+/**
+ * Command for show table partitions Command
+ */
+private[sql] case class ShowCarbonPartitionsCommand(
+    tableIdentifier: TableIdentifier)
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override val output: Seq[Attribute] = CommonUtil.partitionInfoOutput
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(tableIdentifier)(sparkSession).
+      asInstanceOf[CarbonRelation]
+    val carbonTable = relation.tableMeta.carbonTable
+    val tableName = carbonTable.getFactTableName
+    val partitionInfo = carbonTable.getPartitionInfo(
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+    if (partitionInfo == null) {
+      throw new AnalysisException(
+        s"SHOW PARTITIONS is not allowed on a table that is not partitioned: $tableName")
+    }
+    val partitionType = partitionInfo.getPartitionType
+    val columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
+    val LOGGER = LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+    LOGGER.info("partition column name:" + columnName)
+    CommonUtil.getPartitionInfo(columnName, partitionType, partitionInfo)
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
new file mode 100644
index 0000000..6e6a4b1
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.schema
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableColumnSchemaGenerator, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.TableInfo
+import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
+
+private[sql] case class AlterTableAddColumnCommand(
+    alterTableAddColumnsModel: AlterTableAddColumnsModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableAddColumnsModel.tableName
+    val dbName = alterTableAddColumnsModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    var newCols = Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]()
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      // Consider a concurrent scenario where 2 alter operations are executed in parallel. 1st
+      // operation is success and updates the schema file. 2nd operation will get the lock after
+      // completion of 1st operation but as look up relation is called before it will have the
+      // older carbon table and this can lead to inconsistent state in the system. Therefor look
+      // up relation should be called after acquiring the lock
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      // get the latest carbon table and check for column existence
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val schemaConverter = new ThriftWrapperSchemaConverterImpl()
+      val wrapperTableInfo = schemaConverter
+        .fromExternalToWrapperTableInfo(thriftTableInfo,
+          dbName,
+          tableName,
+          carbonTable.getStorePath)
+      newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
+        dbName,
+        wrapperTableInfo,
+        carbonTablePath,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath, sparkSession.sparkContext).process
+      // generate dictionary files for the newly added columns
+      new AlterTableAddColumnRDD(sparkSession.sparkContext,
+        newCols,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath).collect()
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
+      schemaEvolutionEntry.setTimeStamp(timeStamp)
+      schemaEvolutionEntry.setAdded(newCols.toList.asJava)
+      val thriftTable = schemaConverter
+        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
+          thriftTable)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      LOGGER.info(s"Alter table for add columns is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for add columns is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, "Alter table add columns failed")
+        if (newCols.nonEmpty) {
+          LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
+          new AlterTableDropColumnRDD(sparkSession.sparkContext,
+            newCols,
+            carbonTable.getCarbonTableIdentifier,
+            carbonTable.getStorePath).collect()
+          AlterTableUtil.revertAddColumnChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table add operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
new file mode 100644
index 0000000..be87bbb
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
@@ -0,0 +1,116 @@
+/*
+ * 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.schema
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableDataTypeChangeModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
+
+private[sql] case class AlterTableDataTypeChangeCommand(
+    alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableDataTypeChangeModel.tableName
+    val dbName = alterTableDataTypeChangeModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    var locks = List.empty[ICarbonLock]
+    // get the latest carbon table and check for column existence
+    var carbonTable: CarbonTable = null
+    var timeStamp = 0L
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      val columnName = alterTableDataTypeChangeModel.columnName
+      val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala.filter(!_.isInvisible)
+      if (!carbonColumns.exists(_.getColName.equalsIgnoreCase(columnName))) {
+        LOGGER.audit(s"Alter table change data type request has failed. " +
+                     s"Column $columnName does not exist")
+        sys.error(s"Column does not exist: $columnName")
+      }
+      val carbonColumn = carbonColumns.filter(_.getColName.equalsIgnoreCase(columnName))
+      if (carbonColumn.size == 1) {
+        CarbonScalaUtil
+          .validateColumnDataType(alterTableDataTypeChangeModel.dataTypeInfo, carbonColumn.head)
+      } else {
+        LOGGER.audit(s"Alter table change data type request has failed. " +
+                     s"Column $columnName is invalid")
+        sys.error(s"Invalid Column: $columnName")
+      }
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      // maintain the added column for schema evolution history
+      var addColumnSchema: ColumnSchema = null
+      var deletedColumnSchema: ColumnSchema = null
+      val columnSchemaList = tableInfo.fact_table.table_columns.asScala.filter(!_.isInvisible)
+      columnSchemaList.foreach { columnSchema =>
+        if (columnSchema.column_name.equalsIgnoreCase(columnName)) {
+          deletedColumnSchema = columnSchema.deepCopy
+          columnSchema.setData_type(DataTypeConverterUtil
+            .convertToThriftDataType(alterTableDataTypeChangeModel.dataTypeInfo.dataType))
+          columnSchema.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
+          columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
+          addColumnSchema = columnSchema
+        }
+      }
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
+      schemaEvolutionEntry.setAdded(List(addColumnSchema).asJava)
+      schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava)
+      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
+        .setTime_stamp(System.currentTimeMillis)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaEvolutionEntry,
+          tableInfo)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      LOGGER.info(s"Alter table for data type change is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for data type change is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception => LOGGER
+        .error("Alter table change datatype failed : " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil.revertDataTypeChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table data type change operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
new file mode 100644
index 0000000..2f1e3d9
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
@@ -0,0 +1,148 @@
+/*
+ * 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.schema
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.SchemaEvolutionEntry
+import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
+
+private[sql] case class AlterTableDropColumnCommand(
+    alterTableDropColumnModel: AlterTableDropColumnModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableDropColumnModel.tableName
+    val dbName = alterTableDropColumnModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table drop columns request has been received for $dbName.$tableName")
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    // get the latest carbon table and check for column existence
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      val partitionInfo = carbonTable.getPartitionInfo(tableName)
+      if (partitionInfo != null) {
+        val partitionColumnSchemaList = partitionInfo.getColumnSchemaList.asScala
+          .map(_.getColumnName)
+        // check each column existence in the table
+        val partitionColumns = alterTableDropColumnModel.columns.filter {
+          tableColumn => partitionColumnSchemaList.contains(tableColumn)
+        }
+        if (partitionColumns.nonEmpty) {
+          throw new UnsupportedOperationException("Partition columns cannot be dropped: " +
+                                                  s"$partitionColumns")
+        }
+      }
+      val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
+      var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column
+      .ColumnSchema]()
+      var keyColumnCountToBeDeleted = 0
+      // TODO: if deleted column list includes bucketted column throw an error
+      alterTableDropColumnModel.columns.foreach { column =>
+        var columnExist = false
+        tableColumns.foreach { tableColumn =>
+          // column should not be already deleted and should exist in the table
+          if (!tableColumn.isInvisible && column.equalsIgnoreCase(tableColumn.getColName)) {
+            if (tableColumn.isDimension) {
+              keyColumnCountToBeDeleted += 1
+              if (tableColumn.hasEncoding(Encoding.DICTIONARY)) {
+                dictionaryColumns ++= Seq(tableColumn.getColumnSchema)
+              }
+            }
+            columnExist = true
+          }
+        }
+        if (!columnExist) {
+          sys.error(s"Column $column does not exists in the table $dbName.$tableName")
+        }
+      }
+      // take the total key column count. key column to be deleted should not
+      // be >= key columns in schema
+      val totalKeyColumnInSchema = tableColumns.count {
+        tableColumn => !tableColumn.isInvisible && tableColumn.isDimension
+      }
+      if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
+        sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
+      }
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val tableInfo: org.apache.carbondata.format.TableInfo =
+        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      // maintain the deleted columns for schema evolution history
+      var deletedColumnSchema = ListBuffer[org.apache.carbondata.format.ColumnSchema]()
+      val columnSchemaList = tableInfo.fact_table.table_columns.asScala
+      alterTableDropColumnModel.columns.foreach { column =>
+        columnSchemaList.foreach { columnSchema =>
+          if (!columnSchema.invisible && column.equalsIgnoreCase(columnSchema.column_name)) {
+            deletedColumnSchema += columnSchema.deepCopy
+            columnSchema.invisible = true
+          }
+        }
+      }
+      // add deleted columns to schema evolution history and update the schema
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
+      schemaEvolutionEntry.setRemoved(deletedColumnSchema.toList.asJava)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaEvolutionEntry,
+          tableInfo)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      // TODO: 1. add check for deletion of index tables
+      // delete dictionary files for dictionary column and clear dictionary cache from memory
+      new AlterTableDropColumnRDD(sparkSession.sparkContext,
+        dictionaryColumns,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath).collect()
+      LOGGER.info(s"Alter table for drop columns is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for drop columns is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception => LOGGER
+        .error("Alter table drop columns failed : " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil.revertDropColumnChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table drop column operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
new file mode 100644
index 0000000..af361d5
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
@@ -0,0 +1,174 @@
+/*
+ * 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.schema
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command.{AlterTableRenameModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.SchemaEvolutionEntry
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+private[sql] case class AlterTableRenameTableCommand(
+    alterTableRenameModel: AlterTableRenameModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val oldTableIdentifier = alterTableRenameModel.oldTableIdentifier
+    val newTableIdentifier = alterTableRenameModel.newTableIdentifier
+    val oldDatabaseName = oldTableIdentifier.database
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    val newDatabaseName = newTableIdentifier.database
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    if (!oldDatabaseName.equalsIgnoreCase(newDatabaseName)) {
+      throw new MalformedCarbonCommandException("Database name should be same for both tables")
+    }
+    val tableExists = sparkSession.catalog.tableExists(oldDatabaseName, newTableIdentifier.table)
+    if (tableExists) {
+      throw new MalformedCarbonCommandException(s"Table with name $newTableIdentifier " +
+                                                s"already exists")
+    }
+    val oldTableName = oldTableIdentifier.table.toLowerCase
+    val newTableName = newTableIdentifier.table.toLowerCase
+    LOGGER.audit(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
+    LOGGER.info(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val relation: CarbonRelation =
+      metastore.lookupRelation(oldTableIdentifier.database, oldTableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      LOGGER.audit(s"Rename table request has failed. " +
+                   s"Table $oldDatabaseName.$oldTableName does not exist")
+      sys.error(s"Table $oldDatabaseName.$oldTableName does not exist")
+    }
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+      LockUsage.COMPACTION_LOCK,
+      LockUsage.DELETE_SEGMENT_LOCK,
+      LockUsage.CLEAN_FILES_LOCK,
+      LockUsage.DROP_TABLE_LOCK)
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(oldDatabaseName, oldTableName, locksToBeAcquired)(
+          sparkSession)
+      val tableMeta = metastore.lookupRelation(Some(oldDatabaseName), oldTableName)(sparkSession)
+        .asInstanceOf[CarbonRelation].tableMeta
+      carbonTable = tableMeta.carbonTable
+      // get the latest carbon table and check for column existence
+      val carbonTablePath = CarbonStorePath.
+        getCarbonTablePath(AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath))
+      val tableMetadataFile = carbonTablePath.getPath
+      val tableInfo: org.apache.carbondata.format.TableInfo =
+        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
+      schemaEvolutionEntry.setTableName(newTableName)
+      timeStamp = System.currentTimeMillis()
+      schemaEvolutionEntry.setTime_stamp(timeStamp)
+      renameBadRecords(oldTableName, newTableName, oldDatabaseName)
+      val fileType = FileFactory.getFileType(tableMetadataFile)
+      if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
+        val rename = FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
+          .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
+                       newTableName)
+        if (!rename) {
+          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
+          sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
+        }
+      }
+      val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
+        newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
+      val newTablePath = metastore.updateTableSchema(newTableIdentifier,
+        carbonTable.getCarbonTableIdentifier,
+        tableInfo,
+        schemaEvolutionEntry,
+        tableMeta.tablePath)(sparkSession)
+      metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
+      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
+        .runSqlHive(
+          s"ALTER TABLE $oldDatabaseName.$oldTableName RENAME TO $oldDatabaseName.$newTableName")
+      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
+        .runSqlHive(
+          s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
+          s"('tableName'='$newTableName', " +
+          s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
+      sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
+        Some(oldDatabaseName)).quotedString)
+      LOGGER.audit(s"Table $oldTableName has been successfully renamed to $newTableName")
+      LOGGER.info(s"Table $oldTableName has been successfully renamed to $newTableName")
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, "Rename table failed: " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil
+            .revertRenameTableChanges(oldTableIdentifier,
+              newTableName,
+              carbonTable.getStorePath,
+              carbonTable.getCarbonTableIdentifier.getTableId,
+              timeStamp)(
+              sparkSession)
+          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
+        }
+        sys.error(s"Alter table rename table operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+      // case specific to rename table as after table rename old table path will not be found
+      if (carbonTable != null) {
+        AlterTableUtil
+          .releaseLocksManually(locks,
+            locksToBeAcquired,
+            oldDatabaseName,
+            newTableName,
+            carbonTable.getStorePath)
+      }
+    }
+    Seq.empty
+  }
+
+  private def renameBadRecords(
+      oldTableName: String,
+      newTableName: String,
+      dataBaseName: String): Unit = {
+    val oldPath = CarbonUtil
+      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + oldTableName)
+    val newPath = CarbonUtil
+      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + newTableName)
+    val fileType = FileFactory.getFileType(oldPath)
+    if (FileFactory.isFileExist(oldPath, fileType)) {
+      val renameSuccess = FileFactory.getCarbonFile(oldPath, fileType)
+        .renameForce(newPath)
+      if (!renameSuccess) {
+        sys.error(s"BadRecords Folder Rename Failed for table $dataBaseName.$oldTableName")
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
new file mode 100644
index 0000000..c6ca950
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -0,0 +1,554 @@
+/*
+ * 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.strategy
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions.{Attribute, _}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.optimizer.CarbonDecoderRelation
+import org.apache.spark.sql.sources.{BaseRelation, Filter}
+import org.apache.spark.sql.types.{AtomicType, IntegerType, StringType}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.BucketingInfo
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.CarbonAliasDecoderRelation
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Carbon specific optimization 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 {
+  val PUSHED_FILTERS = "PushedFilters"
+
+  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+    plan match {
+      case PhysicalOperation(projects, filters, l: LogicalRelation)
+        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        val relation = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+        pruneFilterProject(
+          l,
+          projects,
+          filters,
+          (a, f, needDecoder) => toCatalystRDD(l, a, relation.buildScan(
+            a.map(_.name).toArray, f), needDecoder)) :: Nil
+      case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
+        if ((profile.isInstanceOf[IncludeProfile] && profile.isEmpty) ||
+            !CarbonDictionaryDecoder.
+              isRequiredToDecode(CarbonDictionaryDecoder.
+                getDictionaryColumnMapping(child.output, relations, profile, aliasMap))) {
+          planLater(child) :: Nil
+        } else {
+          CarbonDictionaryDecoder(relations,
+            profile,
+            aliasMap,
+            planLater(child),
+            SparkSession.getActiveSession.get
+          ) :: Nil
+        }
+      case _ => Nil
+    }
+  }
+
+
+  def getDecoderRDD(
+      logicalRelation: LogicalRelation,
+      projectExprsNeedToDecode: ArrayBuffer[AttributeReference],
+      rdd: RDD[InternalRow],
+      output: Seq[Attribute]): RDD[InternalRow] = {
+    val table = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+    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, Option(table.carbonRelation.tableName))
+      relation.addAttribute(newAttr)
+      newAttr
+    }
+
+    new CarbonDecoderRDD(
+      Seq(relation),
+      IncludeProfile(attrs),
+      CarbonAliasDecoderRelation(),
+      rdd,
+      output,
+      CarbonEnv.getInstance(SparkSession.getActiveSession.get).storePath,
+      table.carbonTable.getTableInfo.serialize())
+  }
+
+  private[this] def toCatalystRDD(
+      relation: LogicalRelation,
+      output: Seq[Attribute],
+      rdd: RDD[InternalRow],
+      needDecode: ArrayBuffer[AttributeReference]):
+  RDD[InternalRow] = {
+    if (needDecode.nonEmpty) {
+      rdd.asInstanceOf[CarbonScanRDD].setVectorReaderSupport(false)
+      getDecoderRDD(relation, needDecode, rdd, output)
+    } else {
+      rdd.asInstanceOf[CarbonScanRDD]
+        .setVectorReaderSupport(supportBatchedDataSource(relation.relation.sqlContext, output))
+      rdd
+    }
+  }
+
+  protected def pruneFilterProject(
+      relation: LogicalRelation,
+      projects: Seq[NamedExpression],
+      filterPredicates: Seq[Expression],
+      scanBuilder: (Seq[Attribute], Array[Filter],
+        ArrayBuffer[AttributeReference]) => RDD[InternalRow]) = {
+    pruneFilterProjectRaw(
+      relation,
+      projects,
+      filterPredicates,
+      (requestedColumns, _, pushedFilters, a) => {
+        scanBuilder(requestedColumns, pushedFilters.toArray, a)
+      })
+  }
+
+  protected def pruneFilterProjectRaw(
+      relation: LogicalRelation,
+      rawProjects: Seq[NamedExpression],
+      filterPredicates: Seq[Expression],
+      scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
+        ArrayBuffer[AttributeReference]) => RDD[InternalRow]) = {
+    val projects = rawProjects.map {p =>
+      p.transform {
+        case CustomDeterministicExpression(exp) => exp
+      }
+    }.asInstanceOf[Seq[NamedExpression]]
+
+    val projectSet = AttributeSet(projects.flatMap(_.references))
+    val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
+
+    val candidatePredicates = filterPredicates.map {
+      _ transform {
+        case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes.
+      }
+    }
+
+    val (unhandledPredicates, pushedFilters) =
+      selectFilters(relation.relation, candidatePredicates)
+
+    // A set of column attributes that are only referenced by pushed down filters.  We can eliminate
+    // them from requested columns.
+    val handledSet = {
+      val handledPredicates = filterPredicates.filterNot(unhandledPredicates.contains)
+      val unhandledSet = AttributeSet(unhandledPredicates.flatMap(_.references))
+      AttributeSet(handledPredicates.flatMap(_.references)) --
+      (projectSet ++ unhandledSet).map(relation.attributeMap)
+    }
+
+    // Combines all Catalyst filter `Expression`s that are either not convertible to data source
+    // `Filter`s or cannot be handled by `relation`.
+    val filterCondition = unhandledPredicates.reduceLeftOption(expressions.And)
+    val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+    val map = table.carbonRelation.metaData.dictionaryMap
+
+    val metadata: Map[String, String] = {
+      val pairs = ArrayBuffer.empty[(String, String)]
+
+      if (pushedFilters.nonEmpty) {
+        pairs += (PUSHED_FILTERS -> pushedFilters.mkString("[", ", ", "]"))
+      }
+      pairs.toMap
+    }
+
+
+    val needDecoder = ArrayBuffer[AttributeReference]()
+    filterCondition match {
+      case Some(exp: Expression) =>
+        exp.references.collect {
+          case attr: AttributeReference =>
+            val dict = map.get(attr.name)
+            if (dict.isDefined && dict.get) {
+              needDecoder += attr
+            }
+        }
+      case None =>
+    }
+
+    projects.map {
+      case attr: AttributeReference =>
+      case Alias(attr: AttributeReference, _) =>
+      case others =>
+        others.references.map { f =>
+          val dictionary = map.get(f.name)
+          if (dictionary.isDefined && dictionary.get) {
+            needDecoder += f.asInstanceOf[AttributeReference]
+          }
+        }
+    }
+
+    if (projects.map(_.toAttribute) == projects &&
+        projectSet.size == projects.size &&
+        filterSet.subsetOf(projectSet)) {
+      // When it is possible to just use column pruning to get the right projection and
+      // when the columns of this projection are enough to evaluate all filter conditions,
+      // just do a scan followed by a filter, with no extra project.
+      val requestedColumns = projects
+        // Safe due to if above.
+        .asInstanceOf[Seq[Attribute]]
+        // Match original case of attributes.
+        .map(relation.attributeMap)
+        // Don't request columns that are only referenced by pushed filters.
+        .filterNot(handledSet.contains)
+      val updateRequestedColumns = updateRequestedColumnsFunc(requestedColumns, table, needDecoder)
+
+      val updateProject = projects.map { expr =>
+        var attr = expr.toAttribute.asInstanceOf[AttributeReference]
+        if (!needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
+          val dict = map.get(attr.name)
+          if (dict.isDefined && dict.get) {
+            attr = AttributeReference(attr.name, IntegerType, attr.nullable, attr.metadata)(attr
+              .exprId, attr.qualifier)
+          }
+        }
+        attr
+      }
+      val scan = getDataSourceScan(relation,
+        updateProject,
+        scanBuilder,
+        candidatePredicates,
+        pushedFilters,
+        metadata,
+        needDecoder,
+        updateRequestedColumns.asInstanceOf[Seq[Attribute]])
+      filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan)
+    } else {
+
+      var newProjectList: Seq[Attribute] = Seq.empty
+      val updatedProjects = projects.map {
+          case a@Alias(s: ScalaUDF, name)
+            if name.equalsIgnoreCase(CarbonCommonConstants.POSITION_ID) ||
+                name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID) =>
+            val reference = AttributeReference(name, StringType, true)().withExprId(a.exprId)
+            newProjectList :+= reference
+            reference
+          case a@Alias(s: ScalaUDF, name)
+            if name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID) =>
+            val reference =
+              AttributeReference(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID,
+                StringType, true)().withExprId(a.exprId)
+            newProjectList :+= reference
+            a.transform {
+              case s: ScalaUDF =>
+                ScalaUDF(s.function, s.dataType, Seq(reference), s.inputTypes)
+            }
+          case other => other
+      }
+      // Don't request columns that are only referenced by pushed filters.
+      val requestedColumns =
+        (projectSet ++ filterSet -- handledSet).map(relation.attributeMap).toSeq ++ newProjectList
+      val updateRequestedColumns = updateRequestedColumnsFunc(requestedColumns, table, needDecoder)
+      val scan = getDataSourceScan(relation,
+        updateRequestedColumns.asInstanceOf[Seq[Attribute]],
+        scanBuilder,
+        candidatePredicates,
+        pushedFilters,
+        metadata,
+        needDecoder,
+        updateRequestedColumns.asInstanceOf[Seq[Attribute]])
+      execution.ProjectExec(
+        updateRequestedColumnsFunc(updatedProjects, table,
+          needDecoder).asInstanceOf[Seq[NamedExpression]],
+        filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan))
+    }
+  }
+
+  def getDataSourceScan(relation: LogicalRelation,
+      output: Seq[Attribute],
+      scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
+        ArrayBuffer[AttributeReference]) => RDD[InternalRow],
+      candidatePredicates: Seq[Expression],
+      pushedFilters: Seq[Filter],
+      metadata: Map[String, String],
+      needDecoder: ArrayBuffer[AttributeReference],
+      updateRequestedColumns: Seq[Attribute]): DataSourceScanExec = {
+    val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+    if (supportBatchedDataSource(relation.relation.sqlContext, updateRequestedColumns) &&
+        needDecoder.isEmpty) {
+      BatchedDataSourceScanExec(
+        output,
+        scanBuilder(updateRequestedColumns, candidatePredicates, pushedFilters, needDecoder),
+        relation.relation,
+        getPartitioning(table.carbonTable, updateRequestedColumns),
+        metadata,
+        relation.catalogTable.map(_.identifier))
+    } else {
+      RowDataSourceScanExec(output,
+        scanBuilder(updateRequestedColumns, candidatePredicates, pushedFilters, needDecoder),
+        relation.relation,
+        getPartitioning(table.carbonTable, updateRequestedColumns),
+        metadata,
+        relation.catalogTable.map(_.identifier))
+    }
+  }
+
+  def updateRequestedColumnsFunc(requestedColumns: Seq[Expression],
+      relation: CarbonDatasourceHadoopRelation,
+      needDecoder: ArrayBuffer[AttributeReference]): Seq[Expression] = {
+    val map = relation.carbonRelation.metaData.dictionaryMap
+    requestedColumns.map {
+      case attr: AttributeReference =>
+        if (needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
+          attr
+        } else {
+          val dict = map.get(attr.name)
+          if (dict.isDefined && dict.get) {
+            AttributeReference(attr.name,
+              IntegerType,
+              attr.nullable,
+              attr.metadata)(attr.exprId, attr.qualifier)
+          } else {
+            attr
+          }
+        }
+      case alias @ Alias(attr: AttributeReference, name) =>
+        if (needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
+          alias
+        } else {
+          val dict = map.get(attr.name)
+          if (dict.isDefined && dict.get) {
+            alias.transform {
+              case attrLocal: AttributeReference =>
+                AttributeReference(attr.name,
+                  IntegerType,
+                  attr.nullable,
+                  attr.metadata)(attr.exprId, attr.qualifier)
+            }
+          } else {
+            alias
+          }
+        }
+      case others => others
+    }
+  }
+
+  private def getPartitioning(carbonTable: CarbonTable,
+      output: Seq[Attribute]): Partitioning = {
+    val info: BucketingInfo = carbonTable.getBucketingInfo(carbonTable.getFactTableName)
+    if (info != null) {
+      val cols = info.getListOfColumns.asScala
+      val sortColumn = carbonTable.
+        getDimensionByTableName(carbonTable.getFactTableName).get(0).getColName
+      val numBuckets = info.getNumberOfBuckets
+      val bucketColumns = cols.flatMap { n =>
+        val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
+        attrRef match {
+          case Some(attr: AttributeReference) =>
+            Some(AttributeReference(attr.name,
+              CarbonScalaUtil.convertCarbonToSparkDataType(n.getDataType),
+              attr.nullable,
+              attr.metadata)(attr.exprId, attr.qualifier))
+          case _ => None
+        }
+      }
+      if (bucketColumns.size == cols.size) {
+        HashPartitioning(bucketColumns, numBuckets)
+      } else {
+        UnknownPartitioning(0)
+      }
+    } else {
+      UnknownPartitioning(0)
+    }
+  }
+
+  protected[sql] def selectFilters(
+      relation: BaseRelation,
+      predicates: Seq[Expression]): (Seq[Expression], Seq[Filter]) = {
+
+    // For conciseness, all Catalyst filter expressions of type `expressions.Expression` below are
+    // called `predicate`s, while all data source filters of type `sources.Filter` are simply called
+    // `filter`s.
+
+    val translated: Seq[(Expression, Filter)] =
+      for {
+        predicate <- predicates
+        filter <- translateFilter(predicate)
+      } yield predicate -> filter
+
+    // A map from original Catalyst expressions to corresponding translated data source filters.
+    val translatedMap: Map[Expression, Filter] = translated.toMap
+
+    // Catalyst predicate expressions that cannot be translated to data source filters.
+    val unrecognizedPredicates = predicates.filterNot(translatedMap.contains)
+
+    // Data source filters that cannot be handled by `relation`. The semantic of a unhandled filter
+    // at here is that a data source may not be able to apply this filter to every row
+    // of the underlying dataset.
+    val unhandledFilters = relation.unhandledFilters(translatedMap.values.toArray).toSet
+
+    val (unhandled, handled) = translated.partition {
+      case (predicate, filter) =>
+        unhandledFilters.contains(filter)
+    }
+
+    // Catalyst predicate expressions that can be translated to data source filters, but cannot be
+    // handled by `relation`.
+    val (unhandledPredicates, _) = unhandled.unzip
+
+    // Translated data source filters that can be handled by `relation`
+    val (_, handledFilters) = handled.unzip
+
+    // translated contains all filters that have been converted to the public Filter interface.
+    // We should always push them to the data source no matter whether the data source can apply
+    // a filter to every row or not.
+    val (_, translatedFilters) = translated.unzip
+
+    (unrecognizedPredicates ++ unhandledPredicates, translatedFilters)
+  }
+
+
+  /**
+   * Tries to translate a Catalyst [[Expression]] into data source [[Filter]].
+   *
+   * @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`.
+   */
+  protected[sql] def translateFilter(predicate: Expression, or: Boolean = false): Option[Filter] = {
+    predicate match {
+      case or@Or(left, right) =>
+
+        val leftFilter = translateFilter(left, true)
+        val rightFilter = translateFilter(right, true)
+        if (leftFilter.isDefined && rightFilter.isDefined) {
+          Some(sources.Or(leftFilter.get, rightFilter.get))
+        } else {
+          None
+        }
+
+      case And(left, right) =>
+        val leftFilter = translateFilter(left, or)
+        val rightFilter = translateFilter(right, or)
+        if (or) {
+          if (leftFilter.isDefined && rightFilter.isDefined) {
+            (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
+          } else {
+            None
+          }
+        } else {
+          (translateFilter(left) ++ translateFilter(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 c@EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case Not(EqualTo(a: Attribute, Literal(v, t))) =>
+        Some(sources.Not(sources.EqualTo(a.name, v)))
+      case Not(EqualTo(Literal(v, t), a: Attribute)) =>
+        Some(sources.Not(sources.EqualTo(a.name, v)))
+      case c@Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      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 c@Not(In(Cast(a: Attribute, _), list))
+        if !list.exists(!_.isInstanceOf[Literal]) =>
+        Some(CastExpr(c))
+      case c@In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
+        Some(CastExpr(c))
+      case InSet(a: Attribute, set) =>
+        Some(sources.In(a.name, set.toArray))
+      case Not(InSet(a: Attribute, set)) =>
+        Some(sources.Not(sources.In(a.name, set.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 c@GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      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 c@LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      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 c@GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      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 c@LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case StartsWith(a: Attribute, Literal(v, t)) =>
+        Some(sources.StringStartsWith(a.name, v.toString))
+      case c@EndsWith(a: Attribute, Literal(v, t)) =>
+        Some(CarbonEndsWith(c))
+      case c@Contains(a: Attribute, Literal(v, t)) =>
+        Some(CarbonContainsWith(c))
+      case others => None
+    }
+  }
+
+  def supportBatchedDataSource(sqlContext: SQLContext, cols: Seq[Attribute]): Boolean = {
+    val vectorizedReader = {
+      if (sqlContext.sparkSession.conf.contains(CarbonCommonConstants.ENABLE_VECTOR_READER)) {
+        sqlContext.sparkSession.conf.get(CarbonCommonConstants.ENABLE_VECTOR_READER)
+      } else if (System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER) != null) {
+        System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER)
+      } else {
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER,
+          CarbonCommonConstants.ENABLE_VECTOR_READER_DEFAULT)
+      }
+    }
+    val supportCodegen =
+      sqlContext.conf.wholeStageEnabled && sqlContext.conf.wholeStageMaxNumFields >= cols.size
+    supportCodegen && vectorizedReader.toBoolean &&
+    cols.forall(_.dataType.isInstanceOf[AtomicType])
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
new file mode 100644
index 0000000..715af1d
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -0,0 +1,162 @@
+/*
+ * 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.strategy
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.management.{AlterTableCompactionCommand, CarbonShowLoadsCommand, LoadTableByInsertCommand, LoadTableCommand}
+import org.apache.spark.sql.execution.command.partition.ShowCarbonPartitionsCommand
+import org.apache.spark.sql.execution.command.schema.{AlterTableAddColumnCommand, AlterTableDataTypeChangeCommand, AlterTableDropColumnCommand, AlterTableRenameTableCommand}
+import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
+
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+/**
+ * Carbon strategies for ddl commands
+ */
+class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
+
+  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+    plan match {
+      case LoadDataCommand(identifier, path, isLocal, isOverwrite, partition)
+        if CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(identifier)(sparkSession) =>
+        ExecutedCommandExec(
+          LoadTableCommand(
+            identifier.database,
+            identifier.table.toLowerCase,
+            path,
+            Seq(),
+            Map(),
+            isOverwrite)) :: Nil
+      case alter@AlterTableRenameCommand(oldTableIdentifier, newTableIdentifier, _) =>
+        val dbOption = oldTableIdentifier.database.map(_.toLowerCase)
+        val tableIdentifier = TableIdentifier(oldTableIdentifier.table.toLowerCase(), dbOption)
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(tableIdentifier)(
+            sparkSession)
+        if (isCarbonTable) {
+          val renameModel = AlterTableRenameModel(tableIdentifier, newTableIdentifier)
+          ExecutedCommandExec(AlterTableRenameTableCommand(renameModel)) :: Nil
+        } else {
+          ExecutedCommandExec(alter) :: Nil
+        }
+      case DropTableCommand(identifier, ifNotExists, isView, _)
+        if CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .isTablePathExists(identifier)(sparkSession) =>
+        ExecutedCommandExec(
+          CarbonDropTableCommand(ifNotExists, identifier.database,
+            identifier.table.toLowerCase)) :: Nil
+      case ShowLoadsCommand(databaseName, table, limit) =>
+        ExecutedCommandExec(
+          CarbonShowLoadsCommand(
+            databaseName,
+            table.toLowerCase,
+            limit,
+            plan.output)) :: Nil
+      case InsertIntoCarbonTable(relation: CarbonDatasourceHadoopRelation,
+      _, child: LogicalPlan, overwrite, _) =>
+        ExecutedCommandExec(LoadTableByInsertCommand(relation, child, overwrite.enabled)) :: Nil
+      case createDb@CreateDatabaseCommand(dbName, ifNotExists, _, _, _) =>
+        CarbonUtil.createDatabaseDirectory(dbName, CarbonEnv.getInstance(sparkSession).storePath)
+        ExecutedCommandExec(createDb) :: Nil
+      case drop@DropDatabaseCommand(dbName, ifExists, isCascade) =>
+        ExecutedCommandExec(CarbonDropDatabaseCommand(drop)) :: Nil
+      case alterTable@AlterTableCompactionCommand(altertablemodel) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(TableIdentifier(altertablemodel.tableName,
+            altertablemodel.dbName))(sparkSession)
+        if (isCarbonTable) {
+          if (altertablemodel.compactionType.equalsIgnoreCase("minor") ||
+              altertablemodel.compactionType.equalsIgnoreCase("major")) {
+            ExecutedCommandExec(alterTable) :: Nil
+          } else {
+            throw new MalformedCarbonCommandException(
+              "Unsupported alter operation on carbon table")
+          }
+        } else {
+          throw new MalformedCarbonCommandException(
+            "Operation not allowed : " + altertablemodel.alterSql)
+        }
+      case dataTypeChange@AlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(TableIdentifier(alterTableChangeDataTypeModel.tableName,
+            alterTableChangeDataTypeModel.databaseName))(sparkSession)
+        if (isCarbonTable) {
+          ExecutedCommandExec(dataTypeChange) :: Nil
+        } else {
+          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
+        }
+      case addColumn@AlterTableAddColumnCommand(alterTableAddColumnsModel) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(TableIdentifier(alterTableAddColumnsModel.tableName,
+            alterTableAddColumnsModel.databaseName))(sparkSession)
+        if (isCarbonTable) {
+          ExecutedCommandExec(addColumn) :: Nil
+        } else {
+          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
+        }
+      case dropColumn@AlterTableDropColumnCommand(alterTableDropColumnModel) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(TableIdentifier(alterTableDropColumnModel.tableName,
+            alterTableDropColumnModel.databaseName))(sparkSession)
+        if (isCarbonTable) {
+          ExecutedCommandExec(dropColumn) :: Nil
+        } else {
+          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
+        }
+      case desc@DescribeTableCommand(identifier, partitionSpec, isExtended, isFormatted)
+        if CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(identifier)(sparkSession) && isFormatted =>
+        val resolvedTable =
+          sparkSession.sessionState.executePlan(UnresolvedRelation(identifier, None)).analyzed
+        val resultPlan = sparkSession.sessionState.executePlan(resolvedTable).executedPlan
+        ExecutedCommandExec(
+          CarbonDescribeFormattedCommand(
+            resultPlan,
+            plan.output,
+            identifier)) :: Nil
+      case ShowPartitionsCommand(t, cols) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(t)(sparkSession)
+        if (isCarbonTable) {
+          ExecutedCommandExec(ShowCarbonPartitionsCommand(t)) :: Nil
+        } else {
+          ExecutedCommandExec(ShowPartitionsCommand(t, cols)) :: Nil
+        }
+      case set@SetCommand(kv) =>
+        ExecutedCommandExec(CarbonSetCommand(set)) :: Nil
+      case reset@ResetCommand =>
+        ExecutedCommandExec(CarbonResetCommand()) :: Nil
+      case org.apache.spark.sql.execution.datasources.CreateTable(tableDesc, mode, None)
+        if tableDesc.provider.get != DDLUtils.HIVE_PROVIDER
+           && tableDesc.provider.get.equals("org.apache.spark.sql.CarbonSource") =>
+        val updatedCatalog =
+          CarbonSource.updateCatalogTableWithCarbonSchema(tableDesc, sparkSession)
+        val cmd =
+          CreateDataSourceTableCommand(updatedCatalog, ignoreIfExists = mode == SaveMode.Ignore)
+        ExecutedCommandExec(cmd) :: Nil
+      case _ => Nil
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index c9fc46c..f61ab84 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -20,12 +20,12 @@ package org.apache.spark.sql.hive
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar}
-import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression}
 import org.apache.spark.sql.catalyst.plans.Inner
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.execution.{ProjectExec, SparkSqlParser, SubqueryExec}
-import org.apache.spark.sql.execution.command.ProjectForDeleteCommand
+import org.apache.spark.sql.execution.SparkSqlParser
+import org.apache.spark.sql.execution.command.mutation.ProjectForDeleteCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -167,13 +167,13 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       updatedSelectPlan
     }
     val tid = CarbonTableIdentifierImplicit.toTableIdentifier(Seq(table.tableIdentifier.toString()))
-    val tidSeq = Seq(getDB.getDatabaseName(tid.database, sparkSession))
+    val tidSeq = Seq(GetDB.getDatabaseName(tid.database, sparkSession))
     val destinationTable = UnresolvedRelation(table.tableIdentifier, table.alias)
     ProjectForUpdate(destinationTable, columns, Seq(finalPlan))
   }
 
   def processDeleteRecordsQuery(selectStmt: String, table: UnresolvedRelation): LogicalPlan = {
-   val tidSeq = Seq(getDB.getDatabaseName(table.tableIdentifier.database, sparkSession),
+   val tidSeq = Seq(GetDB.getDatabaseName(table.tableIdentifier.database, sparkSession),
      table.tableIdentifier.table)
     var addedTupleId = false
     val parsePlan = parser.parsePlan(selectStmt)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
index 1fe6c83..478b178 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -25,16 +25,13 @@ import org.apache.spark.sql.catalyst.expressions.{PredicateSubquery, ScalarSubqu
 import org.apache.spark.sql.catalyst.optimizer.Optimizer
 import org.apache.spark.sql.catalyst.parser.ParserInterface
 import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, SubqueryAlias}
-import org.apache.spark.sql.execution.CarbonLateDecodeStrategy
-import org.apache.spark.sql.execution.command.DDLStrategy
-import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.execution.SparkOptimizer
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
 import org.apache.spark.sql.parser.CarbonSparkSqlParser
 
-import org.apache.carbondata.processing.merger.TableMeta
-
 /**
  * This class will have carbon catalog and refresh the relation from cache if the carbontable in
  * carbon catalog is not same as cached carbon relation's carbon table

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
index f435fa6..1d8bb8a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.execution.command
 
 import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.execution.command._
 
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
@@ -26,7 +27,7 @@ import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
   extends RunnableCommand {
 
-  override val output = command.output
+  override val output: Seq[Attribute] = command.output
 
   override def run(sparkSession: SparkSession): Seq[Row] = {
     val dbName = command.databaseName
@@ -50,7 +51,7 @@ case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
 case class CarbonSetCommand(command: SetCommand)
   extends RunnableCommand {
 
-  override val output = command.output
+  override val output: Seq[Attribute] = command.output
 
   override def run(sparkSession: SparkSession): Seq[Row] = {
     val sessionParms = CarbonEnv.getInstance(sparkSession).carbonSessionInfo.getSessionParams

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
index c6dd905..2ddde7a 100644
--- 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
@@ -27,7 +27,8 @@ 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.{ProjectForUpdateCommand, RunnableCommand}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.execution.command.mutation.ProjectForUpdateCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.types.{IntegerType, StringType}
 


[5/6] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by qi...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
deleted file mode 100644
index 5820b9d..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
+++ /dev/null
@@ -1,857 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command
-
-import java.util
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.storage.StorageLevel
-
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, DeleteDeltaBlockDetails, SegmentUpdateDetails, TupleIdEnum}
-import org.apache.carbondata.core.mutate.data.RowCountDetailsVO
-import org.apache.carbondata.core.statusmanager.{SegmentStatusManager, SegmentUpdateStatusManager}
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
-import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
-import org.apache.carbondata.hadoop.CarbonInputFormat
-import org.apache.carbondata.processing.exception.MultipleMatchingException
-import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CarbonDataMergerUtilResult, CompactionType}
-import org.apache.carbondata.spark.DeleteDelataResultImpl
-import org.apache.carbondata.spark.load.FailureCauses
-import org.apache.carbondata.spark.util.QueryPlanUtil
-
-
-/**
- * IUD update delete and compaction framework.
- *
- */
-
-private[sql] case class ProjectForDeleteCommand(
-     plan: LogicalPlan,
-     identifier: Seq[String],
-     timestamp: String) extends RunnableCommand with DataProcessCommand {
-
-  var horizontalCompactionFailed = false
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val dataFrame = Dataset.ofRows(sparkSession, plan)
-    //    dataFrame.show(truncate = false)
-    //    dataFrame.collect().foreach(println)
-    val dataRdd = dataFrame.rdd
-
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(deleteExecution.getTableIdentifier(identifier))(sparkSession).
-        asInstanceOf[CarbonRelation]
-    val carbonTable = relation.tableMeta.carbonTable
-    val metadataLock = CarbonLockFactory
-        .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-          LockUsage.METADATA_LOCK)
-    var lockStatus = false
-    try {
-      lockStatus = metadataLock.lockWithRetries()
-      LOGGER.audit(s" Delete data request has been received " +
-          s"for ${ relation.databaseName }.${ relation.tableName }.")
-      if (lockStatus) {
-        LOGGER.info("Successfully able to get the table metadata file lock")
-      }
-      else {
-        throw new Exception("Table is locked for deletion. Please try after some time")
-      }
-      val tablePath = CarbonStorePath.getCarbonTablePath(
-        carbonTable.getStorePath,
-        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier)
-      var executorErrors = new ExecutionErrors(FailureCauses.NONE, "")
-
-      // handle the clean up of IUD.
-      CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, false)
-
-      if (deleteExecution
-          .deleteDeltaExecution(identifier, sparkSession, dataRdd, timestamp, relation,
-            false, executorErrors)) {
-        // call IUD Compaction.
-        IUDCommon.tryHorizontalCompaction(sparkSession, relation, isUpdateOperation = false)
-      }
-    } catch {
-      case e: HorizontalCompactionException =>
-        LOGGER.error("Delete operation passed. Exception in Horizontal Compaction." +
-            " Please check logs. " + e.getMessage)
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, e.compactionTimeStamp.toString)
-
-      case e: Exception =>
-        LOGGER.error(e, "Exception in Delete data operation " + e.getMessage)
-        // ****** start clean up.
-        // In case of failure , clean all related delete delta files
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
-
-        // clean up. Null check is required as for executor error some times message is null
-        if (null != e.getMessage) {
-          sys.error("Delete data operation is failed. " + e.getMessage)
-        }
-        else {
-          sys.error("Delete data operation is failed. Please check logs.")
-        }
-    } finally {
-      if (lockStatus) {
-        CarbonLockUtil.fileUnlock(metadataLock, LockUsage.METADATA_LOCK)
-      }
-    }
-    Seq.empty
-  }
-}
-
-private[sql] case class ProjectForUpdateCommand(
-    plan: LogicalPlan, tableIdentifier: Seq[String]) extends RunnableCommand
-    with DataProcessCommand {
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(ProjectForUpdateCommand.getClass.getName)
-
-    //  sqlContext.sparkContext.setLocalProperty(org.apache.spark.sql.execution.SQLExecution
-    //  .EXECUTION_ID_KEY, null)
-    // DataFrame(sqlContext, plan).show(truncate = false)
-    // return Seq.empty
-
-
-    val res = plan find {
-      case relation: LogicalRelation if relation.relation
-          .isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        true
-      case _ => false
-    }
-
-    if (res.isEmpty) {
-      return Seq.empty
-    }
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(deleteExecution.getTableIdentifier(tableIdentifier))(sparkSession).
-        asInstanceOf[CarbonRelation]
-    //    val relation = CarbonEnv.get.carbonMetastore
-    //      .lookupRelation1(deleteExecution.getTableIdentifier(tableIdentifier))(sqlContext).
-    //      asInstanceOf[CarbonRelation]
-    val carbonTable = relation.tableMeta.carbonTable
-    val metadataLock = CarbonLockFactory
-        .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-          LockUsage.METADATA_LOCK)
-    var lockStatus = false
-    // get the current time stamp which should be same for delete and update.
-    val currentTime = CarbonUpdateUtil.readCurrentTime
-    //    var dataFrame: DataFrame = null
-    var dataSet: DataFrame = null
-    var isPersistEnabled = CarbonProperties.getInstance.isPersistUpdateDataset()
-    try {
-      lockStatus = metadataLock.lockWithRetries()
-      if (lockStatus) {
-        logInfo("Successfully able to get the table metadata file lock")
-      }
-      else {
-        throw new Exception("Table is locked for updation. Please try after some time")
-      }
-      val tablePath = CarbonStorePath.getCarbonTablePath(
-        carbonTable.getStorePath,
-        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier)
-      // Get RDD.
-
-      dataSet = if (isPersistEnabled) {
-        Dataset.ofRows(sparkSession, plan).persist(StorageLevel.fromString(
-          CarbonProperties.getInstance.getUpdateDatasetStorageLevel()))
-      }
-      else {
-        Dataset.ofRows(sparkSession, plan)
-      }
-      var executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
-
-
-      // handle the clean up of IUD.
-      CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, false)
-
-      // do delete operation.
-      deleteExecution.deleteDeltaExecution(tableIdentifier, sparkSession, dataSet.rdd,
-        currentTime + "",
-        relation, isUpdateOperation = true, executionErrors)
-
-      if(executionErrors.failureCauses != FailureCauses.NONE) {
-        throw new Exception(executionErrors.errorMsg)
-      }
-
-      // do update operation.
-      UpdateExecution.performUpdate(dataSet, tableIdentifier, plan,
-        sparkSession, currentTime, executionErrors)
-
-      if(executionErrors.failureCauses != FailureCauses.NONE) {
-        throw new Exception(executionErrors.errorMsg)
-      }
-
-      // Do IUD Compaction.
-      IUDCommon.tryHorizontalCompaction(sparkSession, relation, isUpdateOperation = true)
-    } catch {
-      case e: HorizontalCompactionException =>
-        LOGGER.error(
-          "Update operation passed. Exception in Horizontal Compaction. Please check logs." + e)
-        // In case of failure , clean all related delta files
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, e.compactionTimeStamp.toString)
-
-      case e: Exception =>
-        LOGGER.error("Exception in update operation" + e)
-        // ****** start clean up.
-        // In case of failure , clean all related delete delta files
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, currentTime + "")
-
-        // *****end clean up.
-        if (null != e.getMessage) {
-          sys.error("Update operation failed. " + e.getMessage)
-        }
-        if (null != e.getCause && null != e.getCause.getMessage) {
-          sys.error("Update operation failed. " + e.getCause.getMessage)
-        }
-        sys.error("Update operation failed. please check logs.")
-    }
-    finally {
-      if (null != dataSet && isPersistEnabled) {
-        dataSet.unpersist()
-      }
-      if (lockStatus) {
-        CarbonLockUtil.fileUnlock(metadataLock, LockUsage.METADATA_LOCK)
-      }
-    }
-    Seq.empty
-  }
-}
-
-object IUDCommon {
-
-  val LOG = LogServiceFactory.getLogService(this.getClass.getName)
-
-  /**
-   * The method does horizontal compaction. After Update and Delete completion
-   * tryHorizontal compaction will be called. In case this method is called after
-   * Update statement then Update Compaction followed by Delete Compaction will be
-   * processed whereas for tryHorizontalCompaction called after Delete statement
-   * then only Delete Compaction will be processed.
-    *
-    * @param sparkSession
-   * @param carbonRelation
-   * @param isUpdateOperation
-   */
-  def tryHorizontalCompaction(sparkSession: SparkSession,
-      carbonRelation: CarbonRelation,
-      isUpdateOperation: Boolean): Unit = {
-
-    var ishorizontalCompaction = CarbonDataMergerUtil.isHorizontalCompactionEnabled()
-
-    if (ishorizontalCompaction == false) {
-      return
-    }
-
-    var compactionTypeIUD = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
-    val carbonTable = carbonRelation.tableMeta.carbonTable
-    val (db, table) = (carbonTable.getDatabaseName, carbonTable.getFactTableName)
-    val absTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    val updateTimeStamp = System.currentTimeMillis()
-    // To make sure that update and delete timestamps are not same,
-    // required to commit to status metadata and cleanup
-    val deleteTimeStamp = updateTimeStamp + 1
-
-    // get the valid segments
-    var segLists = CarbonDataMergerUtil.getValidSegmentList(absTableIdentifier)
-
-    if (segLists == null || segLists.size() == 0) {
-      return
-    }
-
-    // Should avoid reading Table Status file from Disk every time. Better to load it
-    // in-memory at the starting and pass it along the routines. The constructor of
-    // SegmentUpdateStatusManager reads the Table Status File and Table Update Status
-    // file and save the content in segmentDetails and updateDetails respectively.
-    val segmentUpdateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
-      absTableIdentifier)
-
-    if (isUpdateOperation == true) {
-
-      // This is only update operation, perform only update compaction.
-      compactionTypeIUD = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
-      performUpdateDeltaCompaction(sparkSession,
-        compactionTypeIUD,
-        carbonTable,
-        absTableIdentifier,
-        segmentUpdateStatusManager,
-        updateTimeStamp,
-        segLists)
-    }
-
-    // After Update Compaction perform delete compaction
-    compactionTypeIUD = CompactionType.IUD_DELETE_DELTA_COMPACTION
-    segLists = CarbonDataMergerUtil.getValidSegmentList(absTableIdentifier)
-    if (segLists == null || segLists.size() == 0) {
-      return
-    }
-
-    // Delete Compaction
-    performDeleteDeltaCompaction(sparkSession,
-      compactionTypeIUD,
-      carbonTable,
-      absTableIdentifier,
-      segmentUpdateStatusManager,
-      deleteTimeStamp,
-      segLists)
-  }
-
-  /**
-   * Update Delta Horizontal Compaction.
-    *
-    * @param sparkSession
-   * @param compactionTypeIUD
-   * @param carbonTable
-   * @param absTableIdentifier
-   * @param segLists
-   */
-  private def performUpdateDeltaCompaction(sparkSession: SparkSession,
-      compactionTypeIUD: CompactionType,
-      carbonTable: CarbonTable,
-      absTableIdentifier: AbsoluteTableIdentifier,
-      segmentUpdateStatusManager: SegmentUpdateStatusManager,
-      factTimeStamp: Long,
-      segLists: util.List[String]): Unit = {
-    val db = carbonTable.getDatabaseName
-    val table = carbonTable.getFactTableName
-    // get the valid segments qualified for update compaction.
-    val validSegList = CarbonDataMergerUtil.getSegListIUDCompactionQualified(segLists,
-      absTableIdentifier,
-      segmentUpdateStatusManager,
-      compactionTypeIUD)
-
-    if (validSegList.size() == 0) {
-      return
-    }
-
-    LOG.info(s"Horizontal Update Compaction operation started for [${db}.${table}].")
-    LOG.audit(s"Horizontal Update Compaction operation started for [${db}.${table}].")
-
-    try {
-      // Update Compaction.
-      val altertablemodel = AlterTableModel(Option(carbonTable.getDatabaseName),
-        carbonTable.getFactTableName,
-        Some(segmentUpdateStatusManager),
-        CompactionType.IUD_UPDDEL_DELTA_COMPACTION.toString,
-        Some(factTimeStamp),
-        "")
-
-      AlterTableCompaction(altertablemodel).run(sparkSession)
-    }
-    catch {
-      case e: Exception =>
-        val msg = if (null != e.getMessage) {
-          e.getMessage
-        } else {
-          "Please check logs for more info"
-        }
-        throw new HorizontalCompactionException(
-          s"Horizontal Update Compaction Failed for [${ db }.${ table }]. " + msg, factTimeStamp)
-    }
-    LOG.info(s"Horizontal Update Compaction operation completed for [${ db }.${ table }].")
-    LOG.audit(s"Horizontal Update Compaction operation completed for [${ db }.${ table }].")
-  }
-
-  /**
-   * Delete Delta Horizontal Compaction.
-    *
-    * @param sparkSession
-   * @param compactionTypeIUD
-   * @param carbonTable
-   * @param absTableIdentifier
-   * @param segLists
-   */
-  private def performDeleteDeltaCompaction(sparkSession: SparkSession,
-      compactionTypeIUD: CompactionType,
-      carbonTable: CarbonTable,
-      absTableIdentifier: AbsoluteTableIdentifier,
-      segmentUpdateStatusManager: SegmentUpdateStatusManager,
-      factTimeStamp: Long,
-      segLists: util.List[String]): Unit = {
-
-    val db = carbonTable.getDatabaseName
-    val table = carbonTable.getFactTableName
-    val deletedBlocksList = CarbonDataMergerUtil.getSegListIUDCompactionQualified(segLists,
-      absTableIdentifier,
-      segmentUpdateStatusManager,
-      compactionTypeIUD)
-
-    if (deletedBlocksList.size() == 0) {
-      return
-    }
-
-    LOG.info(s"Horizontal Delete Compaction operation started for [${db}.${table}].")
-    LOG.audit(s"Horizontal Delete Compaction operation started for [${db}.${table}].")
-
-    try {
-
-      // Delete Compaction RDD
-      val rdd1 = sparkSession.sparkContext
-        .parallelize(deletedBlocksList.asScala.toSeq, deletedBlocksList.size())
-
-      val timestamp = factTimeStamp
-      val updateStatusDetails = segmentUpdateStatusManager.getUpdateStatusDetails
-      val result = rdd1.mapPartitions(iter =>
-        new Iterator[Seq[CarbonDataMergerUtilResult]] {
-          override def hasNext: Boolean = iter.hasNext
-
-          override def next(): Seq[CarbonDataMergerUtilResult] = {
-            val segmentAndBlocks = iter.next
-            val segment = segmentAndBlocks.substring(0, segmentAndBlocks.lastIndexOf("/"))
-            val blockName = segmentAndBlocks
-              .substring(segmentAndBlocks.lastIndexOf("/") + 1, segmentAndBlocks.length)
-
-            val result = CarbonDataMergerUtil.compactBlockDeleteDeltaFiles(segment, blockName,
-              absTableIdentifier,
-              updateStatusDetails,
-              timestamp)
-
-            result.asScala.toList
-
-          }
-        }).collect
-
-      val resultList = ListBuffer[CarbonDataMergerUtilResult]()
-      result.foreach(x => {
-        x.foreach(y => {
-          resultList += y
-        })
-      })
-
-      val updateStatus = CarbonDataMergerUtil.updateStatusFile(resultList.toList.asJava,
-        carbonTable,
-        timestamp.toString,
-        segmentUpdateStatusManager)
-      if (updateStatus == false) {
-        LOG.audit(s"Delete Compaction data operation is failed for [${db}.${table}].")
-        LOG.error("Delete Compaction data operation is failed.")
-        throw new HorizontalCompactionException(
-          s"Horizontal Delete Compaction Failed for [${db}.${table}] ." +
-          s" Please check logs for more info.", factTimeStamp)
-      }
-      else {
-        LOG.info(s"Horizontal Delete Compaction operation completed for [${db}.${table}].")
-        LOG.audit(s"Horizontal Delete Compaction operation completed for [${db}.${table}].")
-      }
-    }
-    catch {
-      case e: Exception =>
-        val msg = if (null != e.getMessage) {
-          e.getMessage
-        } else {
-          "Please check logs for more info"
-        }
-        throw new HorizontalCompactionException(
-          s"Horizontal Delete Compaction Failed for [${ db }.${ table }]. " + msg, factTimeStamp)
-    }
-  }
-}
-
-class HorizontalCompactionException(
-    message: String,
-    // required for cleanup
-    val compactionTimeStamp: Long) extends RuntimeException(message) {
-}
-
-object deleteExecution {
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
-  def getTableIdentifier(tableIdentifier: Seq[String]): TableIdentifier = {
-    if (tableIdentifier.size > 1) {
-      TableIdentifier(tableIdentifier(1), Some(tableIdentifier(0)))
-    } else {
-      TableIdentifier(tableIdentifier(0), None)
-    }
-  }
-
-  def deleteDeltaExecution(identifier: Seq[String],
-                           sparkSession: SparkSession,
-                           dataRdd: RDD[Row],
-                           timestamp: String, relation: CarbonRelation, isUpdateOperation: Boolean,
-                           executorErrors: ExecutionErrors): Boolean = {
-
-    var res: Array[List[(String, (SegmentUpdateDetails, ExecutionErrors))]] = null
-    val tableName = getTableIdentifier(identifier).table
-    val database = getDB.getDatabaseName(getTableIdentifier(identifier).database, sparkSession)
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      .lookupRelation(deleteExecution.getTableIdentifier(identifier))(sparkSession).
-      asInstanceOf[CarbonRelation]
-
-    val storeLocation = relation.tableMeta.storePath
-    val absoluteTableIdentifier: AbsoluteTableIdentifier = new
-        AbsoluteTableIdentifier(storeLocation,
-          relation.tableMeta.carbonTableIdentifier)
-    var tablePath = CarbonStorePath
-      .getCarbonTablePath(storeLocation,
-        absoluteTableIdentifier.getCarbonTableIdentifier())
-    var tableUpdateStatusPath = tablePath.getTableUpdateStatusFilePath
-    val totalSegments =
-      SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath).length
-    var factPath = tablePath.getFactDir
-
-    var carbonTable = relation.tableMeta.carbonTable
-    var deleteStatus = true
-    val deleteRdd = if (isUpdateOperation) {
-      val schema =
-        org.apache.spark.sql.types.StructType(Seq(org.apache.spark.sql.types.StructField(
-          CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID,
-          org.apache.spark.sql.types.StringType)))
-      val rdd = dataRdd
-        .map(row => Row(row.get(row.fieldIndex(
-          CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))))
-      sparkSession.createDataFrame(rdd, schema).rdd
-      // sqlContext.createDataFrame(rdd, schema).rdd
-    } else {
-      dataRdd
-    }
-
-    val (carbonInputFormat, job) =
-      QueryPlanUtil.createCarbonInputFormat(absoluteTableIdentifier)
-    CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
-    val keyRdd = deleteRdd.map({ row =>
-      val tupleId: String = row
-        .getString(row.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
-      val key = CarbonUpdateUtil.getSegmentWithBlockFromTID(tupleId)
-      (key, row)
-    }).groupByKey()
-
-    // if no loads are present then no need to do anything.
-    if (keyRdd.partitions.size == 0) {
-      return true
-    }
-
-    var blockMappingVO = carbonInputFormat.getBlockRowCount(job, absoluteTableIdentifier)
-    val segmentUpdateStatusMngr = new SegmentUpdateStatusManager(absoluteTableIdentifier)
-    CarbonUpdateUtil
-      .createBlockDetailsMap(blockMappingVO, segmentUpdateStatusMngr)
-
-    val rowContRdd =
-      sparkSession.sparkContext.parallelize(
-        blockMappingVO.getCompleteBlockRowDetailVO.asScala.toSeq,
-          keyRdd.partitions.length)
-
-//    val rowContRdd = sqlContext.sparkContext
-//      .parallelize(blockMappingVO.getCompleteBlockRowDetailVO.asScala.toSeq,
-//        keyRdd.partitions.size)
-
-    val rdd = rowContRdd.join(keyRdd)
-
-    // rdd.collect().foreach(println)
-
-    res = rdd.mapPartitionsWithIndex(
-      (index: Int, records: Iterator[((String), (RowCountDetailsVO, Iterable[Row]))]) =>
-        Iterator[List[(String, (SegmentUpdateDetails, ExecutionErrors))]] {
-
-          var result = List[(String, (SegmentUpdateDetails, ExecutionErrors))]()
-          while (records.hasNext) {
-            val ((key), (rowCountDetailsVO, groupedRows)) = records.next
-            result = result ++
-              deleteDeltaFunc(index,
-                key,
-                groupedRows.toIterator,
-                timestamp,
-                rowCountDetailsVO)
-
-          }
-          result
-        }
-    ).collect()
-
-    // if no loads are present then no need to do anything.
-    if (res.isEmpty) {
-      return true
-    }
-
-    // update new status file
-    checkAndUpdateStatusFiles
-
-    // all or none : update status file, only if complete delete opeartion is successfull.
-    def checkAndUpdateStatusFiles: Unit = {
-      val blockUpdateDetailsList = new util.ArrayList[SegmentUpdateDetails]()
-      val segmentDetails = new util.HashSet[String]()
-      res.foreach(resultOfSeg => resultOfSeg.foreach(
-        resultOfBlock => {
-          if (resultOfBlock._1.equalsIgnoreCase(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)) {
-            blockUpdateDetailsList.add(resultOfBlock._2._1)
-            segmentDetails.add(resultOfBlock._2._1.getSegmentName)
-            // if this block is invalid then decrement block count in map.
-            if (CarbonUpdateUtil.isBlockInvalid(resultOfBlock._2._1.getStatus)) {
-              CarbonUpdateUtil.decrementDeletedBlockCount(resultOfBlock._2._1,
-                blockMappingVO.getSegmentNumberOfBlockMapping)
-            }
-          }
-          else {
-            deleteStatus = false
-            // In case of failure , clean all related delete delta files
-            CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
-            LOGGER.audit(s"Delete data operation is failed for ${ database }.${ tableName }")
-            val errorMsg =
-              "Delete data operation is failed due to failure in creating delete delta file for " +
-                "segment : " + resultOfBlock._2._1.getSegmentName + " block : " +
-                resultOfBlock._2._1.getBlockName
-            executorErrors.failureCauses = resultOfBlock._2._2.failureCauses
-            executorErrors.errorMsg = resultOfBlock._2._2.errorMsg
-
-            if (executorErrors.failureCauses == FailureCauses.NONE) {
-              executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
-              executorErrors.errorMsg = errorMsg
-            }
-            LOGGER.error(errorMsg)
-            return
-          }
-        }
-      )
-      )
-
-      val listOfSegmentToBeMarkedDeleted = CarbonUpdateUtil
-        .getListOfSegmentsToMarkDeleted(blockMappingVO.getSegmentNumberOfBlockMapping)
-
-
-
-      // this is delete flow so no need of putting timestamp in the status file.
-      if (CarbonUpdateUtil
-        .updateSegmentStatus(blockUpdateDetailsList, carbonTable, timestamp, false) &&
-        CarbonUpdateUtil
-          .updateTableMetadataStatus(segmentDetails,
-            carbonTable,
-            timestamp,
-            !isUpdateOperation,
-            listOfSegmentToBeMarkedDeleted)
-      ) {
-        LOGGER.info(s"Delete data operation is successful for ${ database }.${ tableName }")
-        LOGGER.audit(s"Delete data operation is successful for ${ database }.${ tableName }")
-      }
-      else {
-        // In case of failure , clean all related delete delta files
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
-
-        val errorMessage = "Delete data operation is failed due to failure " +
-          "in table status updation."
-        LOGGER.audit(s"Delete data operation is failed for ${ database }.${ tableName }")
-        LOGGER.error("Delete data operation is failed due to failure in table status updation.")
-        executorErrors.failureCauses = FailureCauses.STATUS_FILE_UPDATION_FAILURE
-        executorErrors.errorMsg = errorMessage
-        // throw new Exception(errorMessage)
-      }
-    }
-
-    def deleteDeltaFunc(index: Int,
-                        key: String,
-                        iter: Iterator[Row],
-                        timestamp: String,
-                        rowCountDetailsVO: RowCountDetailsVO):
-    Iterator[(String, (SegmentUpdateDetails, ExecutionErrors))] = {
-
-      val result = new DeleteDelataResultImpl()
-      var deleteStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
-      val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-      // here key = segment/blockName
-      val blockName = CarbonUpdateUtil
-        .getBlockName(
-          CarbonTablePath.addDataPartPrefix(key.split(CarbonCommonConstants.FILE_SEPARATOR)(1)))
-      val segmentId = key.split(CarbonCommonConstants.FILE_SEPARATOR)(0)
-      var deleteDeltaBlockDetails: DeleteDeltaBlockDetails = new DeleteDeltaBlockDetails(blockName)
-      val resultIter = new Iterator[(String, (SegmentUpdateDetails, ExecutionErrors))] {
-        val segmentUpdateDetails = new SegmentUpdateDetails()
-        var TID = ""
-        var countOfRows = 0
-        try {
-          while (iter.hasNext) {
-            val oneRow = iter.next
-            TID = oneRow
-              .get(oneRow.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID)).toString
-            val offset = CarbonUpdateUtil.getRequiredFieldFromTID(TID, TupleIdEnum.OFFSET)
-            val blockletId = CarbonUpdateUtil
-              .getRequiredFieldFromTID(TID, TupleIdEnum.BLOCKLET_ID)
-            val pageId = Integer.parseInt(CarbonUpdateUtil
-              .getRequiredFieldFromTID(TID, TupleIdEnum.PAGE_ID))
-            val IsValidOffset = deleteDeltaBlockDetails.addBlocklet(blockletId, offset, pageId)
-            // stop delete operation
-            if(!IsValidOffset) {
-              executorErrors.failureCauses = FailureCauses.MULTIPLE_INPUT_ROWS_MATCHING
-              executorErrors.errorMsg = "Multiple input rows matched for same row."
-              throw new MultipleMatchingException("Multiple input rows matched for same row.")
-            }
-            countOfRows = countOfRows + 1
-          }
-
-          val blockPath = CarbonUpdateUtil.getTableBlockPath(TID, factPath)
-          val completeBlockName = CarbonTablePath
-            .addDataPartPrefix(CarbonUpdateUtil.getRequiredFieldFromTID(TID, TupleIdEnum.BLOCK_ID) +
-              CarbonCommonConstants.FACT_FILE_EXT)
-          val deleteDeletaPath = CarbonUpdateUtil
-            .getDeleteDeltaFilePath(blockPath, blockName, timestamp)
-          val carbonDeleteWriter = new CarbonDeleteDeltaWriterImpl(deleteDeletaPath,
-            FileFactory.getFileType(deleteDeletaPath))
-
-
-
-          segmentUpdateDetails.setBlockName(blockName)
-          segmentUpdateDetails.setActualBlockName(completeBlockName)
-          segmentUpdateDetails.setSegmentName(segmentId)
-          segmentUpdateDetails.setDeleteDeltaEndTimestamp(timestamp)
-          segmentUpdateDetails.setDeleteDeltaStartTimestamp(timestamp)
-
-          val alreadyDeletedRows: Long = rowCountDetailsVO.getDeletedRowsInBlock
-          val totalDeletedRows: Long = alreadyDeletedRows + countOfRows
-          segmentUpdateDetails.setDeletedRowsInBlock(totalDeletedRows.toString)
-          if (totalDeletedRows == rowCountDetailsVO.getTotalNumberOfRows) {
-            segmentUpdateDetails.setStatus(CarbonCommonConstants.MARKED_FOR_DELETE)
-          }
-          else {
-            // write the delta file
-            carbonDeleteWriter.write(deleteDeltaBlockDetails)
-          }
-
-          deleteStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
-        } catch {
-          case e : MultipleMatchingException =>
-            LOGGER.audit(e.getMessage)
-            LOGGER.error(e.getMessage)
-          // dont throw exception here.
-          case e: Exception =>
-            val errorMsg = s"Delete data operation is failed for ${ database }.${ tableName }."
-            LOGGER.audit(errorMsg)
-            LOGGER.error(errorMsg + e.getMessage)
-            throw e
-        }
-
-
-        var finished = false
-
-        override def hasNext: Boolean = {
-          if (!finished) {
-            finished = true
-            finished
-          }
-          else {
-            !finished
-          }
-        }
-
-        override def next(): (String, (SegmentUpdateDetails, ExecutionErrors)) = {
-          finished = true
-          result.getKey(deleteStatus, (segmentUpdateDetails, executorErrors))
-        }
-      }
-      resultIter
-    }
-    true
-  }
-}
-
-
-
-object UpdateExecution {
-
-  def performUpdate(
-         dataFrame: Dataset[Row],
-         tableIdentifier: Seq[String],
-         plan: LogicalPlan,
-         sparkSession: SparkSession,
-         currentTime: Long,
-         executorErrors: ExecutionErrors): Unit = {
-
-    def isDestinationRelation(relation: CarbonDatasourceHadoopRelation): Boolean = {
-
-      val tableName = relation.identifier.getCarbonTableIdentifier.getTableName
-      val dbName = relation.identifier.getCarbonTableIdentifier.getDatabaseName
-      (tableIdentifier.size > 1 &&
-        tableIdentifier(0) == dbName &&
-        tableIdentifier(1) == tableName) ||
-        (tableIdentifier(0) == tableName)
-    }
-    def getHeader(relation: CarbonDatasourceHadoopRelation, plan: LogicalPlan): String = {
-      var header = ""
-      var found = false
-
-      plan match {
-        case Project(pList, _) if (!found) =>
-          found = true
-          header = pList
-            .filter(field => !field.name
-              .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
-            .map(col => if (col.name.endsWith(CarbonCommonConstants.UPDATED_COL_EXTENSION)) {
-              col.name
-                .substring(0, col.name.lastIndexOf(CarbonCommonConstants.UPDATED_COL_EXTENSION))
-            }
-            else {
-              col.name
-            }).mkString(",")
-      }
-      header
-    }
-    val ex = dataFrame.queryExecution.analyzed
-    val res = ex find {
-      case relation: LogicalRelation
-        if relation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
-        isDestinationRelation(relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]) =>
-        true
-      case _ => false
-    }
-    val carbonRelation: CarbonDatasourceHadoopRelation = res match {
-      case Some(relation: LogicalRelation) =>
-        relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-      case _ => sys.error("")
-    }
-
-    val updateTableModel = UpdateTableModel(true, currentTime, executorErrors)
-
-    val header = getHeader(carbonRelation, plan)
-
-    LoadTable(
-      Some(carbonRelation.identifier.getCarbonTableIdentifier.getDatabaseName),
-      carbonRelation.identifier.getCarbonTableIdentifier.getTableName,
-      null,
-      Seq(),
-      Map(("fileheader" -> header)),
-      false,
-      null,
-      Some(dataFrame),
-      Some(updateTableModel)).run(sparkSession)
-
-    executorErrors.errorMsg = updateTableModel.executorErrors.errorMsg
-    executorErrors.failureCauses = updateTableModel.executorErrors.failureCauses
-
-    Seq.empty
-
-  }
-
-}