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
-
- }
-
-}