You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2017/11/13 19:21:06 UTC
[1/9] carbondata git commit: [CARBONDATA-1523]Pre Aggregate table
selection and Query Plan changes
Repository: carbondata
Updated Branches:
refs/heads/master 40c31e804 -> e4f2843c6
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
new file mode 100644
index 0000000..3fb0db0
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
@@ -0,0 +1,829 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{AnalysisException, CarbonDatasourceHadoopRelation, InsertIntoCarbonTable, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, Divide, Expression, NamedExpression, PredicateSubquery, ScalaUDF}
+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.datasources.LogicalRelation
+import org.apache.spark.sql.types._
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchema, CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.preagg.{AggregateTableSelector, QueryColumn, QueryPlan}
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Class for applying Pre Aggregate rules
+ * Responsibility.
+ * 1. Check plan is valid plan for updating the parent table plan with child table
+ * 2. Updated the plan based on child schema
+ *
+ * Rules for Upadating the plan
+ * 1. Grouping expression rules
+ * 1.1 Change the parent attribute reference for of group expression
+ * to child attribute reference
+ *
+ * 2. Aggregate expression rules
+ * 2.1 Change the parent attribute reference for of group expression to
+ * child attribute reference
+ * 2.2 Change the count AggregateExpression to Sum as count
+ * is already calculated so in case of aggregate table
+ * we need to apply sum to get the count
+ * 2.2 In case of average aggregate function select 2 columns from aggregate table with
+ * aggregation
+ * sum and count. Then add divide(sum(column with sum), sum(column with count)).
+ * Note: During aggregate table creation for average table will be created with two columns
+ * one for sum(column) and count(column) to support rollup
+ *
+ * 3. Filter Expression rules.
+ * 3.1 Updated filter expression attributes with child table attributes
+ * 4. Update the Parent Logical relation with child Logical relation
+ *
+ * @param sparkSession
+ * spark session
+ */
+case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule[LogicalPlan] {
+
+ override def apply(plan: LogicalPlan): LogicalPlan = {
+ var needAnalysis = true
+ plan.transformExpressions {
+ // first check if any preAgg scala function is applied it is present is in plan
+ // then call is from create preaggregate table class so no need to transform the query plan
+ case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase("preAgg") =>
+ needAnalysis = false
+ al
+ // in case of query if any unresolve alias is present then wait for plan to be resolved
+ // return the same plan as we can tranform the plan only when everything is resolved
+ case unresolveAlias@UnresolvedAlias(_, _) =>
+ needAnalysis = false
+ unresolveAlias
+ case attr@UnresolvedAttribute(_) =>
+ needAnalysis = false
+ attr
+ }
+ // if plan is not valid for transformation then return same plan
+ if (!needAnalysis) {
+ plan
+ } else {
+ // create buffer to collect all the column and its metadata information
+ val list = scala.collection.mutable.ListBuffer.empty[QueryColumn]
+ var isValidPlan = true
+ val carbonTable = plan match {
+ // matching the plan based on supported plan
+ // if plan is matches with any case it will validate and get all
+ // information required for transforming the plan
+
+ // When plan has grouping expression, aggregate expression
+ // subquery
+ case Aggregate(groupingExp,
+ aggregateExp,
+ SubqueryAlias(_, logicalRelation: LogicalRelation, _))
+ // only carbon query plan is supported checking whether logical relation is
+ // is for carbon
+ if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+ logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
+ .hasDataMapSchema =>
+ val (carbonTable, tableName) = getCarbonTableAndTableName(logicalRelation)
+ // if it is valid plan then extract the query columns
+ isValidPlan = extractQueryColumnsFromAggExpression(groupingExp,
+ aggregateExp,
+ carbonTable,
+ tableName,
+ list)
+ carbonTable
+
+ // below case for handling filter query
+ // When plan has grouping expression, aggregate expression
+ // filter expression
+ case Aggregate(groupingExp, aggregateExp,
+ Filter(filterExp,
+ SubqueryAlias(_, logicalRelation: LogicalRelation, _)))
+ // only carbon query plan is supported checking whether logical relation is
+ // is for carbon
+ if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+ logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
+ .hasDataMapSchema =>
+ val (carbonTable, tableName) = getCarbonTableAndTableName(logicalRelation)
+ // if it is valid plan then extract the query columns
+ isValidPlan = extractQueryColumnsFromAggExpression(groupingExp,
+ aggregateExp,
+ carbonTable,
+ tableName,
+ list)
+ // TODO need to handle filter predicate subquery scenario
+ isValidPlan = !PredicateSubquery.hasPredicateSubquery(filterExp)
+ // getting the columns from filter expression
+ if(isValidPlan) {
+ filterExp.transform {
+ case attr: AttributeReference =>
+ list += getQueryColumn(attr.name, carbonTable, tableName, isFilterColumn = true)
+ attr
+ }
+ }
+ carbonTable
+
+ // When plan has grouping expression, aggregate expression
+ // logical relation
+ case Aggregate(groupingExp, aggregateExp, logicalRelation: LogicalRelation)
+ // only carbon query plan is supported checking whether logical relation is
+ // is for carbon
+ if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+ logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable
+ .hasDataMapSchema =>
+ val (carbonTable, tableName) = getCarbonTableAndTableName(logicalRelation)
+ // if it is valid plan then extract the query columns
+ isValidPlan = extractQueryColumnsFromAggExpression(groupingExp,
+ aggregateExp,
+ carbonTable,
+ tableName,
+ list)
+ carbonTable
+ case _ =>
+ isValidPlan = false
+ null
+ }
+ // if plan is valid then update the plan with child attributes
+ if (isValidPlan) {
+ // getting all the projection columns
+ val listProjectionColumn = list
+ .filter(queryColumn => queryColumn.getAggFunction.isEmpty && !queryColumn.isFilterColumn)
+ // getting all the filter columns
+ val listFilterColumn = list
+ .filter(queryColumn => queryColumn.getAggFunction.isEmpty && queryColumn.isFilterColumn)
+ // getting all the aggregation columns
+ val listAggregationColumn = list.filter(queryColumn => !queryColumn.getAggFunction.isEmpty)
+ // create a query plan object which will be used to select the list of pre aggregate tables
+ // matches with this plan
+ val queryPlan = new QueryPlan(listProjectionColumn.asJava,
+ listAggregationColumn.asJava,
+ listFilterColumn.asJava)
+ // create aggregate table selector object
+ val aggregateTableSelector = new AggregateTableSelector(queryPlan, carbonTable)
+ // select the list of valid child tables
+ val selectedDataMapSchemas = aggregateTableSelector.selectPreAggDataMapSchema()
+ // if it doesnot match with any pre aggregate table return the same plan
+ if (!selectedDataMapSchemas.isEmpty) {
+ // sort the selected child schema based on size to select smallest pre aggregate table
+ val (aggDataMapSchema, carbonRelation) =
+ selectedDataMapSchemas.asScala.map { selectedDataMapSchema =>
+ val catalog = sparkSession.sessionState.catalog
+ val carbonRelation = catalog
+ .lookupRelation(TableIdentifier(selectedDataMapSchema.getRelationIdentifier
+ .getTableName,
+ Some(selectedDataMapSchema.getRelationIdentifier
+ .getDatabaseName))).asInstanceOf[SubqueryAlias].child
+ .asInstanceOf[LogicalRelation]
+ (selectedDataMapSchema, carbonRelation)
+ }.minBy(f => f._2.relation.asInstanceOf[CarbonDatasourceHadoopRelation].sizeInBytes)
+ // transform the query plan based on selected child schema
+ transformPreAggQueryPlan(plan, aggDataMapSchema, carbonRelation)
+ } else {
+ plan
+ }
+ } else {
+ plan
+ }
+ }
+ }
+
+ /**
+ * Below method will be used to get the child attribute reference
+ * based on parent name
+ *
+ * @param dataMapSchema
+ * child schema
+ * @param attributeReference
+ * parent attribute reference
+ * @param childCarbonRelation
+ * child logical relation
+ * @param aggFunction
+ * aggregation function applied on child
+ * @return child attribute reference
+ */
+ def getChildAttributeReference(dataMapSchema: DataMapSchema,
+ attributeReference: AttributeReference,
+ childCarbonRelation: LogicalRelation,
+ aggFunction: String = ""): AttributeReference = {
+ val aggregationDataMapSchema = dataMapSchema.asInstanceOf[AggregationDataMapSchema];
+ val columnSchema = if (aggFunction.isEmpty) {
+ aggregationDataMapSchema.getChildColByParentColName(attributeReference.name)
+ } else {
+ aggregationDataMapSchema.getAggChildColByParent(attributeReference.name, aggFunction)
+ }
+ // here column schema cannot be null, if it is null then aggregate table selection
+ // logic has some problem
+ if (null == columnSchema) {
+ throw new AnalysisException("Column doesnot exists in Pre Aggregate table")
+ }
+ // finding the child attribute from child logical relation
+ childCarbonRelation.attributeMap.find(p => p._2.name.equals(columnSchema.getColumnName)).get._2
+ }
+
+ /**
+ * Below method will be used to transform the main table plan to child table plan
+ * rules for transformming is as below.
+ * 1. Grouping expression rules
+ * 1.1 Change the parent attribute reference for of group expression
+ * to child attribute reference
+ *
+ * 2. Aggregate expression rules
+ * 2.1 Change the parent attribute reference for of group expression to
+ * child attribute reference
+ * 2.2 Change the count AggregateExpression to Sum as count
+ * is already calculated so in case of aggregate table
+ * we need to apply sum to get the count
+ * 2.2 In case of average aggregate function select 2 columns from aggregate table with
+ * aggregation sum and count. Then add divide(sum(column with sum), sum(column with count)).
+ * Note: During aggregate table creation for average table will be created with two columns
+ * one for sum(column) and count(column) to support rollup
+ * 3. Filter Expression rules.
+ * 3.1 Updated filter expression attributes with child table attributes
+ * 4. Update the Parent Logical relation with child Logical relation
+ *
+ * @param logicalPlan
+ * parent logical plan
+ * @param aggDataMapSchema
+ * select data map schema
+ * @param childCarbonRelation
+ * child carbon table relation
+ * @return transformed plan
+ */
+ def transformPreAggQueryPlan(logicalPlan: LogicalPlan,
+ aggDataMapSchema: DataMapSchema, childCarbonRelation: LogicalRelation): LogicalPlan = {
+ logicalPlan.transform {
+ case Aggregate(grExp, aggExp, child@SubqueryAlias(_, l: LogicalRelation, _))
+ if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+ l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable.hasDataMapSchema =>
+ val (updatedGroupExp, updatedAggExp, newChild, None) =
+ getUpdatedExpressions(grExp,
+ aggExp,
+ child,
+ None,
+ aggDataMapSchema,
+ childCarbonRelation)
+ Aggregate(updatedGroupExp,
+ updatedAggExp,
+ newChild)
+ case Aggregate(grExp,
+ aggExp,
+ Filter(expression, child@SubqueryAlias(_, l: LogicalRelation, _)))
+ if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+ l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable.hasDataMapSchema =>
+ val (updatedGroupExp, updatedAggExp, newChild, updatedFilterExpression) =
+ getUpdatedExpressions(grExp,
+ aggExp,
+ child,
+ Some(expression),
+ aggDataMapSchema,
+ childCarbonRelation)
+ Aggregate(updatedGroupExp,
+ updatedAggExp,
+ Filter(updatedFilterExpression.get,
+ newChild))
+ case Aggregate(grExp, aggExp, l: LogicalRelation)
+ if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+ l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable.hasDataMapSchema =>
+ val (updatedGroupExp, updatedAggExp, newChild, None) =
+ getUpdatedExpressions(grExp,
+ aggExp,
+ l,
+ None,
+ aggDataMapSchema,
+ childCarbonRelation)
+ Aggregate(updatedGroupExp,
+ updatedAggExp,
+ newChild)
+ }
+ }
+
+ /**
+ * Below method will be used to get the updated expression for pre aggregated table.
+ * It will replace the attribute of actual plan with child table attributes.
+ * Updation will be done for below expression.
+ * 1. Grouping expression
+ * 2. aggregate expression
+ * 3. child logical plan
+ * 4. filter expression if present
+ *
+ * @param groupingExpressions
+ * actual plan grouping expression
+ * @param aggregateExpressions
+ * actual plan aggregate expression
+ * @param child
+ * child logical plan
+ * @param filterExpression
+ * filter expression
+ * @param aggDataMapSchema
+ * pre aggregate table schema
+ * @param childCarbonRelation
+ * pre aggregate table logical relation
+ * @return tuple of(updated grouping expression,
+ * updated aggregate expression,
+ * updated child logical plan,
+ * updated filter expression if present in actual plan)
+ */
+ def getUpdatedExpressions(groupingExpressions: Seq[Expression],
+ aggregateExpressions: Seq[NamedExpression],
+ child: LogicalPlan, filterExpression: Option[Expression] = None,
+ aggDataMapSchema: DataMapSchema,
+ childCarbonRelation: LogicalRelation): (Seq[Expression], Seq[NamedExpression], LogicalPlan,
+ Option[Expression]) = {
+ // transforming the group by expression attributes with child attributes
+ val updatedGroupExp = groupingExpressions.map { exp =>
+ exp.transform {
+ case attr: AttributeReference =>
+ getChildAttributeReference(aggDataMapSchema, attr, childCarbonRelation)
+ }
+ }
+ // below code is for updating the aggregate expression.
+ // Note: In case of aggregate expression updation we need to return alias as
+ // while showing the final result we need to show based on actual query
+ // for example: If query is "select name from table group by name"
+ // if we only update the attributes it will show child table column name in final output
+ // so for handling this if attributes does not have alias we need to return alias of
+ // parent
+ // table column name
+ // Rules for updating aggregate expression.
+ // 1. If it matches with attribute reference return alias of child attribute reference
+ // 2. If it matches with alias return same alias with child attribute reference
+ // 3. If it matches with alias of any supported aggregate function return aggregate function
+ // with child attribute reference. Please check class level documentation how when aggregate
+ // function will be updated
+
+ val updatedAggExp = aggregateExpressions.map {
+ // case for attribute reference
+ case attr: AttributeReference =>
+ val childAttributeReference = getChildAttributeReference(aggDataMapSchema,
+ attr,
+ childCarbonRelation)
+ // returning the alias to show proper column name in output
+ Alias(childAttributeReference,
+ attr.name)(NamedExpression.newExprId,
+ childAttributeReference.qualifier).asInstanceOf[NamedExpression]
+ // case for alias
+ case Alias(attr: AttributeReference, name) =>
+ val childAttributeReference = getChildAttributeReference(aggDataMapSchema,
+ attr,
+ childCarbonRelation)
+ // returning alias with child attribute reference
+ Alias(childAttributeReference,
+ name)(NamedExpression.newExprId,
+ childAttributeReference.qualifier).asInstanceOf[NamedExpression]
+ // for aggregate function case
+ case alias@Alias(attr: AggregateExpression, name) =>
+ // get the updated aggregate aggregate function
+ val aggExp = getUpdatedAggregateExpressionForChild(attr,
+ aggDataMapSchema,
+ childCarbonRelation)
+ // returning alias with child attribute reference
+ Alias(aggExp,
+ name)(NamedExpression.newExprId,
+ alias.qualifier).asInstanceOf[NamedExpression]
+ }
+ // transformaing the logical relation
+ val newChild = child.transform {
+ case _: LogicalRelation =>
+ childCarbonRelation
+ case _: SubqueryAlias =>
+ childCarbonRelation
+ }
+ // updating the filter expression if present
+ val updatedFilterExpression = if (filterExpression.isDefined) {
+ val filterExp = filterExpression.get
+ Some(filterExp.transform {
+ case attr: AttributeReference =>
+ getChildAttributeReference(aggDataMapSchema, attr, childCarbonRelation)
+ })
+ } else {
+ None
+ }
+ (updatedGroupExp, updatedAggExp, newChild, updatedFilterExpression)
+ }
+
+ /**
+ * Below method will be used to get the aggregate expression based on match
+ * Aggregate expression updation rules
+ * 1 Change the count AggregateExpression to Sum as count
+ * is already calculated so in case of aggregate table
+ * we need to apply sum to get the count
+ * 2 In case of average aggregate function select 2 columns from aggregate table
+ * with aggregation sum and count.
+ * Then add divide(sum(column with sum), sum(column with count)).
+ * Note: During aggregate table creation for average aggregation function
+ * table will be created with two columns one for sum(column) and count(column)
+ * to support rollup
+ *
+ * @param aggExp
+ * aggregate expression
+ * @param dataMapSchema
+ * child data map schema
+ * @param childCarbonRelation
+ * child logical relation
+ * @return updated expression
+ */
+ def getUpdatedAggregateExpressionForChild(aggExp: AggregateExpression,
+ dataMapSchema: DataMapSchema,
+ childCarbonRelation: LogicalRelation):
+ Expression = {
+ aggExp.aggregateFunction match {
+ // Change the count AggregateExpression to Sum as count
+ // is already calculated so in case of aggregate table
+ // we need to apply sum to get the count
+ case count@Count(Seq(attr: AttributeReference)) =>
+ AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ count.prettyName),
+ LongType)),
+ aggExp.mode,
+ isDistinct = false)
+ case sum@Sum(attr: AttributeReference) =>
+ AggregateExpression(Sum(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ sum.prettyName)),
+ aggExp.mode,
+ isDistinct = false)
+ case max@Max(attr: AttributeReference) =>
+ AggregateExpression(Max(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ max.prettyName)),
+ aggExp.mode,
+ isDistinct = false)
+ case min@Min(attr: AttributeReference) =>
+ AggregateExpression(Min(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ min.prettyName)),
+ aggExp.mode,
+ isDistinct = false)
+ case sum@Sum(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ sum.prettyName),
+ changeDataType)),
+ aggExp.mode,
+ isDistinct = false)
+ case min@Min(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ AggregateExpression(Min(Cast(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ min.prettyName),
+ changeDataType)),
+ aggExp.mode,
+ isDistinct = false)
+ case max@Max(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ AggregateExpression(Max(Cast(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ max.prettyName),
+ changeDataType)),
+ aggExp.mode,
+ isDistinct = false)
+
+ // In case of average aggregate function select 2 columns from aggregate table
+ // with aggregation sum and count.
+ // Then add divide(sum(column with sum), sum(column with count)).
+ case Average(attr: AttributeReference) =>
+ Divide(AggregateExpression(Sum(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ "sum")),
+ aggExp.mode,
+ isDistinct = false),
+ AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ "count"),
+ LongType)),
+ aggExp.mode,
+ isDistinct = false))
+ // In case of average aggregate function select 2 columns from aggregate table
+ // with aggregation sum and count.
+ // Then add divide(sum(column with sum), sum(column with count)).
+ case Average(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ Divide(AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ "sum"),
+ changeDataType)),
+ aggExp.mode,
+ isDistinct = false),
+ AggregateExpression(Sum(Cast(getChildAttributeReference(dataMapSchema,
+ attr,
+ childCarbonRelation,
+ "count"),
+ LongType)),
+ aggExp.mode,
+ isDistinct = false))
+ }
+ }
+
+ /**
+ * Method to get the carbon table and table name
+ *
+ * @param parentLogicalRelation
+ * parent table relation
+ * @return tuple of carbon table and table name
+ */
+ def getCarbonTableAndTableName(parentLogicalRelation: LogicalRelation): (CarbonTable, String) = {
+ val carbonTable = parentLogicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+ .carbonRelation
+ .metaData.carbonTable
+ val tableName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+ .getTableName
+ (carbonTable, tableName)
+ }
+
+ /**
+ * Below method will be used to get the query columns from plan
+ *
+ * @param groupByExpression
+ * group by expression
+ * @param aggregateExpressions
+ * aggregate expression
+ * @param carbonTable
+ * parent carbon table
+ * @param tableName
+ * parent table name
+ * @param list
+ * list of attributes
+ * @return plan is valid
+ */
+ def extractQueryColumnsFromAggExpression(groupByExpression: Seq[Expression],
+ aggregateExpressions: Seq[NamedExpression],
+ carbonTable: CarbonTable, tableName: String,
+ list: scala.collection.mutable.ListBuffer[QueryColumn]): Boolean = {
+ aggregateExpressions.map {
+ case attr: AttributeReference =>
+ list += getQueryColumn(attr.name,
+ carbonTable,
+ tableName);
+ case Alias(attr: AttributeReference, _) =>
+ list += getQueryColumn(attr.name,
+ carbonTable,
+ tableName);
+ case Alias(attr: AggregateExpression, _) =>
+ if (attr.isDistinct) {
+ return false
+ }
+ val queryColumn = validateAggregateFunctionAndGetFields(carbonTable,
+ attr.aggregateFunction,
+ tableName)
+ if (queryColumn.nonEmpty) {
+ list ++= queryColumn
+ } else {
+ return false
+ }
+ }
+ true
+ }
+
+ /**
+ * Below method will be used to validate aggregate function and get the attribute information
+ * which is applied on select query.
+ * Currently sum, max, min, count, avg is supported
+ * in case of any other aggregate function it will return empty sequence
+ * In case of avg it will return two fields one for count
+ * and other of sum of that column to support rollup
+ *
+ * @param carbonTable
+ * parent table
+ * @param aggFunctions
+ * aggregation function
+ * @param tableName
+ * parent table name
+ * @return list of fields
+ */
+ def validateAggregateFunctionAndGetFields(carbonTable: CarbonTable,
+ aggFunctions: AggregateFunction,
+ tableName: String
+ ): Seq[QueryColumn] = {
+ val changedDataType = true
+ aggFunctions match {
+ case sum@Sum(attr: AttributeReference) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ sum.prettyName))
+ case sum@Sum(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ sum.prettyName,
+ changeDataType.typeName,
+ changedDataType))
+ case count@Count(Seq(attr: AttributeReference)) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ count.prettyName))
+ case min@Min(attr: AttributeReference) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ min.prettyName))
+ case min@Min(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ min.prettyName,
+ changeDataType.typeName,
+ changedDataType))
+ case max@Max(attr: AttributeReference) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ max.prettyName))
+ case max@Max(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ max.prettyName,
+ changeDataType.typeName,
+ changedDataType))
+ // in case of average need to return two columns
+ // sum and count of the column to added during table creation to support rollup
+ case Average(attr: AttributeReference) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ "sum"
+ ), getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ "count"
+ ))
+ // in case of average need to return two columns
+ // sum and count of the column to added during table creation to support rollup
+ case Average(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ Seq(getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ "sum",
+ changeDataType.typeName,
+ changedDataType), getQueryColumn(attr.name,
+ carbonTable,
+ tableName,
+ "count",
+ changeDataType.typeName,
+ changedDataType))
+ case _ =>
+ Seq.empty
+ }
+ }
+
+ /**
+ * Below method will be used to get the query column object which
+ * will have details of the column and its property
+ *
+ * @param columnName
+ * parent column name
+ * @param carbonTable
+ * parent carbon table
+ * @param tableName
+ * parent table name
+ * @param aggFunction
+ * aggregate function applied
+ * @param dataType
+ * data type of the column
+ * @param isChangedDataType
+ * is cast is applied on column
+ * @param isFilterColumn
+ * is filter is applied on column
+ * @return query column
+ */
+ def getQueryColumn(columnName: String,
+ carbonTable: CarbonTable,
+ tableName: String,
+ aggFunction: String = "",
+ dataType: String = "",
+ isChangedDataType: Boolean = false,
+ isFilterColumn: Boolean = false): QueryColumn = {
+ val columnSchema = carbonTable.getColumnByName(tableName, columnName).getColumnSchema
+ if (isChangedDataType) {
+ new QueryColumn(columnSchema, columnSchema.getDataType.getName, aggFunction, isFilterColumn)
+ } else {
+ new QueryColumn(columnSchema,
+ CarbonScalaUtil.convertSparkToCarbonSchemaDataType(dataType),
+ aggFunction, isFilterColumn)
+ }
+ }
+}
+
+/**
+ * Insert into carbon table from other source
+ */
+object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
+ def apply(plan: LogicalPlan): LogicalPlan = {
+ plan.transform {
+ // Wait until children are resolved.
+ case p: LogicalPlan if !p.childrenResolved => p
+
+ case p@InsertIntoTable(relation: LogicalRelation, _, child, _, _)
+ if relation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+ castChildOutput(p, relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation], child)
+ }
+ }
+
+ def castChildOutput(p: InsertIntoTable,
+ relation: CarbonDatasourceHadoopRelation,
+ child: LogicalPlan)
+ : LogicalPlan = {
+ if (relation.carbonRelation.output.size > CarbonCommonConstants
+ .DEFAULT_MAX_NUMBER_OF_COLUMNS) {
+ sys
+ .error("Maximum supported column by carbon is:" + CarbonCommonConstants
+ .DEFAULT_MAX_NUMBER_OF_COLUMNS
+ )
+ }
+ val isAggregateTable = !relation.carbonRelation.tableMeta.carbonTable.getTableInfo
+ .getParentRelationIdentifiers.isEmpty
+ // transform logical plan if the load is for aggregate table.
+ val childPlan = if (isAggregateTable) {
+ transformAggregatePlan(child)
+ } else {
+ child
+ }
+ if (childPlan.output.size >= relation.carbonRelation.output.size) {
+ val newChildOutput = childPlan.output.zipWithIndex.map { columnWithIndex =>
+ columnWithIndex._1 match {
+ case attr: Alias =>
+ Alias(attr.child, s"col${ columnWithIndex._2 }")(attr.exprId)
+ case attr: Attribute =>
+ Alias(attr, s"col${ columnWithIndex._2 }")(NamedExpression.newExprId)
+ case attr => attr
+ }
+ }
+ val newChild: LogicalPlan = if (newChildOutput == childPlan.output) {
+ p.child
+ } else {
+ Project(newChildOutput, childPlan)
+ }
+ InsertIntoCarbonTable(relation, p.partition, newChild, p.overwrite, p.ifNotExists)
+ } else {
+ sys.error("Cannot insert into target table because column number are different")
+ }
+ }
+
+ /**
+ * Transform the logical plan with average(col1) aggregation type to sum(col1) and count(col1).
+ *
+ * @param logicalPlan
+ * @return
+ */
+ private def transformAggregatePlan(logicalPlan: LogicalPlan): LogicalPlan = {
+ logicalPlan transform {
+ case aggregate@Aggregate(_, aExp, _) =>
+ val newExpressions = aExp.flatMap {
+ case alias@Alias(attrExpression: AggregateExpression, _) =>
+ attrExpression.aggregateFunction match {
+ case Average(attr: AttributeReference) =>
+ Seq(Alias(attrExpression
+ .copy(aggregateFunction = Sum(attr),
+ resultId = NamedExpression.newExprId), attr.name + "_sum")(),
+ Alias(attrExpression
+ .copy(aggregateFunction = Count(attr),
+ resultId = NamedExpression.newExprId), attr.name + "_count")())
+ case Average(cast@Cast(attr: AttributeReference, _)) =>
+ Seq(Alias(attrExpression
+ .copy(aggregateFunction = Sum(cast),
+ resultId = NamedExpression.newExprId),
+ attr.name + "_sum")(),
+ Alias(attrExpression
+ .copy(aggregateFunction = Count(cast),
+ resultId = NamedExpression.newExprId), attr.name + "_count")())
+ case _ => Seq(alias)
+ }
+ case namedExpr: NamedExpression => Seq(namedExpr)
+ }
+ aggregate.copy(aggregateExpressions = newExpressions.asInstanceOf[Seq[NamedExpression]])
+ case plan: LogicalPlan => plan
+ }
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 f698dd4..205b716 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
@@ -169,15 +169,15 @@ class CarbonSessionState(sparkSession: SparkSession) extends HiveSessionState(sp
catalog.ParquetConversions ::
catalog.OrcConversions ::
CarbonPreInsertionCasts ::
+ CarbonPreAggregateQueryRules(sparkSession) ::
CarbonIUDAnalysisRule(sparkSession) ::
AnalyzeCreateTable(sparkSession) ::
PreprocessTableInsertion(conf) ::
DataSourceAnalysis(conf) ::
(if (conf.runSQLonFile) {
new ResolveDataSource(sparkSession) :: Nil
- } else {
- Nil
- })
+ } else { Nil }
+ )
override val extendedCheckRules = Seq(
PreWriteCheck(conf, catalog))
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 46a2515..3bed9d1 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
@@ -476,6 +476,14 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
}
}
+ def addPreAggFunction(sql: String): String = {
+ addPreAgg(new lexical.Scanner(sql.toLowerCase)) match {
+ case Success(query, _) => query
+ case failureOrError => throw new MalformedCarbonCommandException(
+ s"Unsupported query")
+ }
+ }
+
def getBucketFields(
properties: mutable.Map[String, String],
fields: Seq[Field],
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 5c51156..401b149 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
@@ -18,7 +18,7 @@ package org.apache.spark.sql.parser
import scala.collection.mutable
-import org.apache.spark.sql.{CarbonSession, DataFrame, Dataset, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, CarbonSession, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, ParseException, SqlBaseParser}
import org.apache.spark.sql.catalyst.parser.ParserUtils._
@@ -74,6 +74,7 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
val parser = new CarbonSpark2SqlParser
override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = {
+ Option(ctx.query()).map(plan)
val fileStorage = Option(ctx.createFileFormat) match {
case Some(value) =>
if (value.children.get(1).getText.equalsIgnoreCase("by")) {
[2/9] carbondata git commit: [CARBONDATA-1523]Pre Aggregate table
selection and Query Plan changes
Posted by ra...@apache.org.
[CARBONDATA-1523]Pre Aggregate table selection and Query Plan changes
This closes #1464
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/e4f2843c
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/e4f2843c
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/e4f2843c
Branch: refs/heads/master
Commit: e4f2843c6d9aa3dc8dfca52c6fcfa084b07492a5
Parents: 1c16afa
Author: kumarvishal <ku...@gmail.com>
Authored: Mon Oct 30 12:44:32 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530
----------------------------------------------------------------------
.../core/constants/CarbonCommonConstants.java | 2 +
.../ThriftWrapperSchemaConverterImpl.java | 7 +-
.../schema/table/AggregationDataMapSchema.java | 212 +++++
.../core/metadata/schema/table/CarbonTable.java | 16 +-
.../metadata/schema/table/DataMapSchema.java | 19 +-
.../schema/table/DataMapSchemaFactory.java | 38 +
.../core/metadata/schema/table/TableInfo.java | 7 +-
.../core/metadata/schema/table/TableSchema.java | 5 +-
.../core/preagg/AggregateTableSelector.java | 135 +++
.../carbondata/core/preagg/QueryColumn.java | 70 ++
.../carbondata/core/preagg/QueryPlan.java | 59 ++
.../TestPreAggregateTableSelection.scala | 175 ++++
.../spark/sql/catalyst/CarbonDDLSqlParser.scala | 7 +
.../spark/rdd/CarbonDataRDDFactory.scala | 2 +-
.../scala/org/apache/spark/sql/CarbonEnv.scala | 3 +
.../CreatePreAggregateTableCommand.scala | 12 +-
.../preaaggregate/PreAggregateListeners.scala | 24 +-
.../preaaggregate/PreAggregateUtil.scala | 184 ++--
.../spark/sql/hive/CarbonAnalysisRules.scala | 101 +--
.../sql/hive/CarbonPreAggregateRules.scala | 829 +++++++++++++++++++
.../spark/sql/hive/CarbonSessionState.scala | 6 +-
.../sql/parser/CarbonSpark2SqlParser.scala | 8 +
.../spark/sql/parser/CarbonSparkSqlParser.scala | 3 +-
23 files changed, 1709 insertions(+), 215 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index e27e5bd..8018c2b 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1416,6 +1416,8 @@ public final class CarbonCommonConstants {
public static final String CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT = "true";
+ public static final String AGGREGATIONDATAMAPSCHEMA = "AggregateDataMapHandler";
+
private CarbonCommonConstants() {
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index b914e06..fef2e0f 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -31,6 +31,7 @@ import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchemaFactory;
import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
import org.apache.carbondata.core.metadata.schema.table.TableInfo;
import org.apache.carbondata.core.metadata.schema.table.TableSchema;
@@ -628,10 +629,10 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
@Override public DataMapSchema fromExternalToWrapperDataMapSchema(
org.apache.carbondata.format.DataMapSchema thriftDataMapSchema) {
- DataMapSchema childSchema =
- new DataMapSchema(thriftDataMapSchema.getDataMapName(), thriftDataMapSchema.getClassName());
+ DataMapSchema childSchema = DataMapSchemaFactory.INSTANCE
+ .getDataMapSchema(thriftDataMapSchema.getDataMapName(), thriftDataMapSchema.getClassName());
childSchema.setProperties(thriftDataMapSchema.getProperties());
- if (thriftDataMapSchema.getRelationIdentifire() != null) {
+ if (null != thriftDataMapSchema.getRelationIdentifire()) {
RelationIdentifier relationIdentifier =
new RelationIdentifier(thriftDataMapSchema.getRelationIdentifire().getDatabaseName(),
thriftDataMapSchema.getRelationIdentifire().getTableName(),
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java
new file mode 100644
index 0000000..87c07f4
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/AggregationDataMapSchema.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.metadata.schema.table;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
+
+/**
+ * data map schema class for pre aggregation
+ */
+public class AggregationDataMapSchema extends DataMapSchema {
+
+ /**
+ * map of parent column name to set of child column column without
+ * aggregation function
+ */
+ private Map<String, Set<ColumnSchema>> parentToNonAggChildMapping;
+
+ /**
+ * map of parent column name to set of child columns column with
+ * aggregation function
+ */
+ private Map<String, Set<ColumnSchema>> parentToAggChildMapping;
+
+ /**
+ * map of parent column name to set of aggregation function applied in
+ * in parent column
+ */
+ private Map<String, Set<String>> parentColumnToAggregationsMapping;
+
+ public AggregationDataMapSchema(String dataMapName, String className) {
+ super(dataMapName, className);
+ }
+
+ public void setChildSchema(TableSchema childSchema) {
+ super.setChildSchema(childSchema);
+ List<ColumnSchema> listOfColumns = getChildSchema().getListOfColumns();
+ fillNonAggFunctionColumns(listOfColumns);
+ fillAggFunctionColumns(listOfColumns);
+ fillParentNameToAggregationMapping(listOfColumns);
+ }
+
+ /**
+ * Below method will be used to get the columns on which aggregate function is not applied
+ * @param columnName
+ * parent column name
+ * @return child column schema
+ */
+ public ColumnSchema getNonAggChildColBasedByParent(String columnName) {
+ Set<ColumnSchema> columnSchemas = parentToNonAggChildMapping.get(columnName);
+ if (null != columnSchemas) {
+ Iterator<ColumnSchema> iterator = columnSchemas.iterator();
+ while (iterator.hasNext()) {
+ ColumnSchema next = iterator.next();
+ if (null == next.getAggFunction() || next.getAggFunction().isEmpty()) {
+ return next;
+ }
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Below method will be used to get the column schema based on parent column name
+ * @param columName
+ * parent column name
+ * @return child column schema
+ */
+ public ColumnSchema getChildColByParentColName(String columName) {
+ List<ColumnSchema> listOfColumns = childSchema.getListOfColumns();
+ for (ColumnSchema columnSchema : listOfColumns) {
+ List<ParentColumnTableRelation> parentColumnTableRelations =
+ columnSchema.getParentColumnTableRelations();
+ if (parentColumnTableRelations.get(0).getColumnName().equals(columName)) {
+ return columnSchema;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Below method will be used to get the child column schema based on parent name and aggregate
+ * function applied on column
+ * @param columnName
+ * parent column name
+ * @param aggFunction
+ * aggregate function applied
+ * @return child column schema
+ */
+ public ColumnSchema getAggChildColByParent(String columnName,
+ String aggFunction) {
+ Set<ColumnSchema> columnSchemas = parentToAggChildMapping.get(columnName);
+ if (null != columnSchemas) {
+ Iterator<ColumnSchema> iterator = columnSchemas.iterator();
+ while (iterator.hasNext()) {
+ ColumnSchema next = iterator.next();
+ if (null != next.getAggFunction() && next.getAggFunction().equalsIgnoreCase(aggFunction)) {
+ return next;
+ }
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Below method is to check if parent column with matching aggregate function
+ * @param parentColumnName
+ * parent column name
+ * @param aggFunction
+ * aggregate function
+ * @return is matching
+ */
+ public boolean isColumnWithAggFunctionExists(String parentColumnName, String aggFunction) {
+ Set<String> aggFunctions = parentColumnToAggregationsMapping.get(parentColumnName);
+ if (null != aggFunctions && aggFunctions.contains(aggFunction)) {
+ return true;
+ }
+ return false;
+ }
+
+
+ /**
+ * Method to prepare mapping of parent to list of aggregation function applied on that column
+ * @param listOfColumns
+ * child column schema list
+ */
+ private void fillParentNameToAggregationMapping(List<ColumnSchema> listOfColumns) {
+ parentColumnToAggregationsMapping = new HashMap<>();
+ for (ColumnSchema column : listOfColumns) {
+ if (null != column.getAggFunction() && !column.getAggFunction().isEmpty()) {
+ List<ParentColumnTableRelation> parentColumnTableRelations =
+ column.getParentColumnTableRelations();
+ if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1) {
+ String columnName = column.getParentColumnTableRelations().get(0).getColumnName();
+ Set<String> aggFunctions = parentColumnToAggregationsMapping.get(columnName);
+ if (null == aggFunctions) {
+ aggFunctions = new HashSet<>();
+ parentColumnToAggregationsMapping.put(columnName, aggFunctions);
+ }
+ aggFunctions.add(column.getAggFunction());
+ }
+ }
+ }
+ }
+
+ /**
+ * Below method will be used prepare mapping between parent column to non aggregation function
+ * columns
+ * @param listOfColumns
+ * list of child columns
+ */
+ private void fillNonAggFunctionColumns(List<ColumnSchema> listOfColumns) {
+ parentToNonAggChildMapping = new HashMap<>();
+ for (ColumnSchema column : listOfColumns) {
+ if (null == column.getAggFunction() || column.getAggFunction().isEmpty()) {
+ fillMappingDetails(column, parentToNonAggChildMapping);
+ }
+ }
+ }
+
+ private void fillMappingDetails(ColumnSchema column,
+ Map<String, Set<ColumnSchema>> map) {
+ List<ParentColumnTableRelation> parentColumnTableRelations =
+ column.getParentColumnTableRelations();
+ if (null != parentColumnTableRelations && parentColumnTableRelations.size() == 1) {
+ String columnName = column.getParentColumnTableRelations().get(0).getColumnName();
+ Set<ColumnSchema> columnSchemas = map.get(columnName);
+ if (null == columnSchemas) {
+ columnSchemas = new HashSet<>();
+ map.put(columnName, columnSchemas);
+ }
+ columnSchemas.add(column);
+ }
+ }
+
+ /**
+ * Below method will be used to fill parent to list of aggregation column mapping
+ * @param listOfColumns
+ * list of child columns
+ */
+ private void fillAggFunctionColumns(List<ColumnSchema> listOfColumns) {
+ parentToAggChildMapping = new HashMap<>();
+ for (ColumnSchema column : listOfColumns) {
+ if (null != column.getAggFunction() && !column.getAggFunction().isEmpty()) {
+ fillMappingDetails(column, parentToAggChildMapping);
+ }
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index ca0952d..0fd9fbf 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -126,6 +126,8 @@ public class CarbonTable implements Serializable {
private int dimensionOrdinalMax;
+ private boolean hasDataMapSchema;
+
private CarbonTable() {
this.tableDimensionsMap = new HashMap<String, List<CarbonDimension>>();
this.tableImplicitDimensionsMap = new HashMap<String, List<CarbonDimension>>();
@@ -158,6 +160,8 @@ public class CarbonTable implements Serializable {
table.tablePartitionMap.put(tableInfo.getFactTable().getTableName(),
tableInfo.getFactTable().getPartitionInfo());
}
+ table.hasDataMapSchema =
+ null != tableInfo.getDataMapSchemaList() && tableInfo.getDataMapSchemaList().size() > 0;
return table;
}
@@ -702,13 +706,13 @@ public class CarbonTable implements Serializable {
this.dimensionOrdinalMax = dimensionOrdinalMax;
}
- public boolean isPreAggregateTable() {
- return tableInfo.getParentRelationIdentifiers() != null && !tableInfo
- .getParentRelationIdentifiers().isEmpty();
+
+ public boolean hasDataMapSchema() {
+ return hasDataMapSchema;
}
- public boolean hasPreAggregateTables() {
- return tableInfo.getDataMapSchemaList() != null && !tableInfo
- .getDataMapSchemaList().isEmpty();
+ public boolean isChildDataMap() {
+ return null != tableInfo.getParentRelationIdentifiers()
+ && !tableInfo.getParentRelationIdentifiers().isEmpty();
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
index e0632d9..5a9017b 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
@@ -30,20 +30,20 @@ public class DataMapSchema implements Serializable, Writable {
private static final long serialVersionUID = 6577149126264181553L;
- private String dataMapName;
+ protected String dataMapName;
private String className;
- private RelationIdentifier relationIdentifier;
+ protected RelationIdentifier relationIdentifier;
/**
* child table schema
*/
- private TableSchema childSchema;
+ protected TableSchema childSchema;
/**
* relation properties
*/
- private Map<String, String> properties;
+ protected Map<String, String> properties;
public DataMapSchema() {
}
@@ -69,6 +69,10 @@ public class DataMapSchema implements Serializable, Writable {
return properties;
}
+ public String getDataMapName() {
+ return dataMapName;
+ }
+
public void setRelationIdentifier(RelationIdentifier relationIdentifier) {
this.relationIdentifier = relationIdentifier;
}
@@ -81,10 +85,6 @@ public class DataMapSchema implements Serializable, Writable {
this.properties = properties;
}
- public String getDataMapName() {
- return dataMapName;
- }
-
@Override public void write(DataOutput out) throws IOException {
out.writeUTF(dataMapName);
out.writeUTF(className);
@@ -114,7 +114,7 @@ public class DataMapSchema implements Serializable, Writable {
this.className = in.readUTF();
boolean isRelationIdnentifierExists = in.readBoolean();
if (isRelationIdnentifierExists) {
- this.relationIdentifier = new RelationIdentifier();
+ this.relationIdentifier = new RelationIdentifier(null, null, null);
this.relationIdentifier.readFields(in);
}
boolean isChildSchemaExists = in.readBoolean();
@@ -130,6 +130,5 @@ public class DataMapSchema implements Serializable, Writable {
String value = in.readUTF();
this.properties.put(key, value);
}
-
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
new file mode 100644
index 0000000..5729959
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchemaFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.metadata.schema.table;
+
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.AGGREGATIONDATAMAPSCHEMA;
+
+public class DataMapSchemaFactory {
+ public static final DataMapSchemaFactory INSTANCE = new DataMapSchemaFactory();
+
+ /**
+ * Below class will be used to get data map schema object
+ * based on class name
+ * @param className
+ * @return data map schema
+ */
+ public DataMapSchema getDataMapSchema(String dataMapName, String className) {
+ switch (className) {
+ case AGGREGATIONDATAMAPSCHEMA:
+ return new AggregationDataMapSchema(dataMapName, className);
+ default:
+ return new DataMapSchema(dataMapName, className);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index 1d9e2ec..65878bc 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -295,7 +295,12 @@ public class TableInfo implements Serializable, Writable {
for (int i = 0; i < numberOfChildTable; i++) {
DataMapSchema childSchema = new DataMapSchema();
childSchema.readFields(in);
- dataMapSchemaList.add(childSchema);
+ DataMapSchema dataMapSchema = DataMapSchemaFactory.INSTANCE
+ .getDataMapSchema(childSchema.getDataMapName(), childSchema.getClassName());
+ dataMapSchema.setChildSchema(childSchema.getChildSchema());
+ dataMapSchema.setRelationIdentifier(childSchema.getRelationIdentifier());
+ dataMapSchema.setProperties(childSchema.getProperties());
+ dataMapSchemaList.add(dataMapSchema);
}
}
boolean isParentTableRelationIndentifierExists = in.readBoolean();
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
index 714e0d8..03848d9 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
@@ -263,9 +263,10 @@ public class TableSchema implements Serializable, Writable {
Map<String, String> properties = new HashMap<>();
properties.put("CHILD_SELECT QUERY", queryString);
properties.put("QUERYTYPE", queryType);
- DataMapSchema dataMapSchema = new DataMapSchema(dataMapName, className);
- dataMapSchema.setChildSchema(this);
+ DataMapSchema dataMapSchema =
+ new DataMapSchema(dataMapName, className);
dataMapSchema.setProperties(properties);
+ dataMapSchema.setChildSchema(this);
dataMapSchema.setRelationIdentifier(relationIdentifier);
return dataMapSchema;
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java b/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java
new file mode 100644
index 0000000..8b87a1a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/preagg/AggregateTableSelector.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.preagg;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.metadata.schema.table.AggregationDataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Below class will be used to select the aggregate table based
+ * query plan. Rules for selecting the aggregate table is below:
+ * 1. Select all aggregate table based on projection
+ * 2. select aggregate table based on filter exp,
+ * 2. select if aggregate tables based on aggregate columns
+ */
+public class AggregateTableSelector {
+
+ /**
+ * current query plan
+ */
+ private QueryPlan queryPlan;
+
+ /**
+ * parent table
+ */
+ private CarbonTable parentTable;
+
+ public AggregateTableSelector(QueryPlan queryPlan, CarbonTable parentTable) {
+ this.queryPlan = queryPlan;
+ this.parentTable = parentTable;
+ }
+
+ /**
+ * Below method will be used to select pre aggregate tables based on query plan
+ * Rules for selecting the aggregate table is below:
+ * 1. Select all aggregate table based on projection
+ * 2. select aggregate table based on filter exp,
+ * 2. select if aggregate tables based on aggregate columns
+ *
+ * @return selected pre aggregate table schema
+ */
+ public List<DataMapSchema> selectPreAggDataMapSchema() {
+ List<QueryColumn> projectionColumn = queryPlan.getProjectionColumn();
+ List<QueryColumn> aggColumns = queryPlan.getAggregationColumns();
+ List<QueryColumn> filterColumns = queryPlan.getFilterColumns();
+ List<DataMapSchema> dataMapSchemaList = parentTable.getTableInfo().getDataMapSchemaList();
+ List<DataMapSchema> selectedDataMapSchema = new ArrayList<>();
+ boolean isMatch;
+ // match projection columns
+ if (null != projectionColumn && !projectionColumn.isEmpty()) {
+ for (DataMapSchema dmSchema : dataMapSchemaList) {
+ AggregationDataMapSchema aggregationDataMapSchema = (AggregationDataMapSchema) dmSchema;
+ isMatch = true;
+ for (QueryColumn queryColumn : projectionColumn) {
+ ColumnSchema columnSchemaByParentName = aggregationDataMapSchema
+ .getNonAggChildColBasedByParent(queryColumn.getColumnSchema().getColumnName());
+ if (null == columnSchemaByParentName) {
+ isMatch = false;
+ }
+ }
+ if (isMatch) {
+ selectedDataMapSchema.add(dmSchema);
+ }
+ }
+ // if projection column is present but selected table list size is zero then
+ if (selectedDataMapSchema.size() == 0) {
+ return selectedDataMapSchema;
+ }
+ }
+
+ // match filter columns
+ if (null != filterColumns && !filterColumns.isEmpty()) {
+ List<DataMapSchema> dmSchemaToIterate =
+ selectedDataMapSchema.isEmpty() ? dataMapSchemaList : selectedDataMapSchema;
+ selectedDataMapSchema = new ArrayList<>();
+ for (DataMapSchema dmSchema : dmSchemaToIterate) {
+ isMatch = true;
+ for (QueryColumn queryColumn : filterColumns) {
+ AggregationDataMapSchema aggregationDataMapSchema = (AggregationDataMapSchema) dmSchema;
+ ColumnSchema columnSchemaByParentName = aggregationDataMapSchema
+ .getNonAggChildColBasedByParent(queryColumn.getColumnSchema().getColumnName());
+ if (null == columnSchemaByParentName) {
+ isMatch = false;
+ }
+ }
+ if (isMatch) {
+ selectedDataMapSchema.add(dmSchema);
+ }
+ }
+ // if filter column is present and selection size is zero then return
+ if (selectedDataMapSchema.size() == 0) {
+ return selectedDataMapSchema;
+ }
+ }
+ // match aggregation columns
+ if (null != aggColumns && !aggColumns.isEmpty()) {
+ List<DataMapSchema> dmSchemaToIterate =
+ selectedDataMapSchema.isEmpty() ? dataMapSchemaList : selectedDataMapSchema;
+ selectedDataMapSchema = new ArrayList<>();
+ for (DataMapSchema dmSchema : dmSchemaToIterate) {
+ isMatch = true;
+ for (QueryColumn queryColumn : aggColumns) {
+ AggregationDataMapSchema aggregationDataMapSchema = (AggregationDataMapSchema) dmSchema;
+ if (!aggregationDataMapSchema
+ .isColumnWithAggFunctionExists(queryColumn.getColumnSchema().getColumnName(),
+ queryColumn.getAggFunction())) {
+ isMatch = false;
+ }
+ }
+ if (isMatch) {
+ selectedDataMapSchema.add(dmSchema);
+ }
+ }
+ }
+ return selectedDataMapSchema;
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java b/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java
new file mode 100644
index 0000000..a62d556
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/preagg/QueryColumn.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.preagg;
+
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * column present in query
+ */
+public class QueryColumn {
+
+ /**
+ * parent column schema
+ */
+ private ColumnSchema columnSchema;
+
+ /**
+ * to store the change data type in case of cast
+ */
+ private String changedDataType;
+
+ /**
+ * aggregation function applied
+ */
+ private String aggFunction;
+
+ /**
+ * is filter column
+ */
+ private boolean isFilterColumn;
+
+ public QueryColumn(ColumnSchema columnSchema, String changedDataType, String aggFunction,
+ boolean isFilterColumn) {
+ this.columnSchema = columnSchema;
+ this.changedDataType = changedDataType;
+ this.aggFunction = aggFunction;
+ this.isFilterColumn = isFilterColumn;
+ }
+
+ public ColumnSchema getColumnSchema() {
+ return columnSchema;
+ }
+
+ public String getChangedDataType() {
+ return changedDataType;
+ }
+
+ public String getAggFunction() {
+ return aggFunction;
+ }
+
+ public boolean isFilterColumn() {
+ return isFilterColumn;
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/core/src/main/java/org/apache/carbondata/core/preagg/QueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/preagg/QueryPlan.java b/core/src/main/java/org/apache/carbondata/core/preagg/QueryPlan.java
new file mode 100644
index 0000000..21a34fa
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/preagg/QueryPlan.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.preagg;
+
+import java.util.List;
+
+/**
+ * class to maintain the query plan to select the data map tables
+ */
+public class QueryPlan {
+
+ /**
+ * List of projection columns
+ */
+ private List<QueryColumn> projectionColumn;
+
+ /**
+ * list of aggregation columns
+ */
+ private List<QueryColumn> aggregationColumns;
+
+ /**
+ * list of filter columns
+ */
+ private List<QueryColumn> filterColumns;
+
+ public QueryPlan(List<QueryColumn> projectionColumn, List<QueryColumn> aggregationColumns,
+ List<QueryColumn> filterColumns) {
+ this.projectionColumn = projectionColumn;
+ this.aggregationColumns = aggregationColumns;
+ this.filterColumns = filterColumns;
+ }
+
+ public List<QueryColumn> getProjectionColumn() {
+ return projectionColumn;
+ }
+
+ public List<QueryColumn> getAggregationColumns() {
+ return aggregationColumns;
+ }
+
+ public List<QueryColumn> getFilterColumns() {
+ return filterColumns;
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
new file mode 100644
index 0000000..6b435c6
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateTableSelection.scala
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.integration.spark.testsuite.preaTable1regate
+
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, DataFrame}
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestPreAggregateTableSelection extends QueryTest with BeforeAndAfterAll {
+
+ override def beforeAll: Unit = {
+ sql("drop table if exists mainTable")
+ sql("drop table if exists agg0")
+ sql("drop table if exists agg1")
+ sql("drop table if exists agg2")
+ sql("drop table if exists agg3")
+ sql("drop table if exists agg4")
+ sql("drop table if exists agg5")
+ sql("drop table if exists agg6")
+ sql("drop table if exists agg7")
+ sql("CREATE TABLE mainTable(id int, name string, city string, age string) STORED BY 'org.apache.carbondata.format'")
+ sql("create datamap agg0 on table mainTable using 'preaggregate' as select name from mainTable group by name")
+ sql("create datamap agg1 on table mainTable using 'preaggregate' as select name,sum(age) from mainTable group by name")
+ sql("create datamap agg2 on table mainTable using 'preaggregate' as select name,sum(id) from mainTable group by name")
+ sql("create datamap agg3 on table mainTable using 'preaggregate' as select name,count(id) from mainTable group by name")
+ sql("create datamap agg4 on table mainTable using 'preaggregate' as select name,sum(age),count(id) from mainTable group by name")
+ sql("create datamap agg5 on table mainTable using 'preaggregate' as select name,avg(age) from mainTable group by name")
+ sql("create datamap agg6 on table mainTable using 'preaggregate' as select name,min(age) from mainTable group by name")
+ sql("create datamap agg7 on table mainTable using 'preaggregate' as select name,max(age) from mainTable group by name")
+ sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table mainTable")
+ }
+
+
+ test("test PreAggregate table selection 1") {
+ val df = sql("select name from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
+ }
+
+ test("test PreAggregate table selection 2") {
+ val df = sql("select name from mainTable where name in (select name from mainTable) group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "mainTable")
+ }
+
+ test("test PreAggregate table selection 3") {
+ val df = sql("select name from mainTable where name in (select name from mainTable group by name) group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "mainTable")
+ }
+
+ test("test PreAggregate table selection 4") {
+ val df = sql("select name from mainTable where name in('vishal') group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg0")
+ }
+
+ test("test PreAggregate table selection 5") {
+ val df = sql("select name, sum(age) from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
+ }
+
+ test("test PreAggregate table selection 6") {
+ val df = sql("select sum(age) from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
+ }
+
+ test("test PreAggregate table selection 7") {
+ val df = sql("select sum(id) from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg2")
+ }
+
+ test("test PreAggregate table selection 8") {
+ val df = sql("select count(id) from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg3")
+ }
+
+ test("test PreAggregate table selection 9") {
+ val df = sql("select sum(age), count(id) from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg4")
+ }
+
+ test("test PreAggregate table selection 10") {
+ val df = sql("select avg(age) from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg5")
+ }
+
+ test("test PreAggregate table selection 11") {
+ val df = sql("select max(age) from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg7")
+ }
+
+ test("test PreAggregate table selection 12") {
+ val df = sql("select min(age) from mainTable group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg6")
+ }
+
+ test("test PreAggregate table selection 13") {
+ val df = sql("select name, sum(age) from mainTable where city = 'Bangalore' group by name")
+ preAggTableValidator(df.queryExecution.analyzed, "mainTable")
+ }
+
+ test("test PreAggregate table selection 14") {
+ val df = sql("select sum(age) from mainTable")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg1")
+ }
+
+ test("test PreAggregate table selection 15") {
+ val df = sql("select avg(age) from mainTable")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg5")
+ }
+
+ test("test PreAggregate table selection 16") {
+ val df = sql("select max(age) from mainTable")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg7")
+ }
+
+ test("test PreAggregate table selection 17") {
+ val df = sql("select min(age) from mainTable")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg6")
+ }
+
+ test("test PreAggregate table selection 18") {
+ val df = sql("select count(id) from mainTable")
+ preAggTableValidator(df.queryExecution.analyzed, "maintable_agg3")
+ }
+
+ def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit ={
+ var isValidPlan = false
+ plan.transform {
+ // first check if any preaTable1 scala function is applied it is present is in plan
+ // then call is from create preaTable1regate table class so no need to transform the query plan
+ case logicalRelation:LogicalRelation =>
+ if(logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
+ val relation = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+ if(relation.carbonTable.getFactTableName.equalsIgnoreCase(actualTableName)) {
+ isValidPlan = true
+ }
+ }
+ logicalRelation
+ }
+ if(!isValidPlan) {
+ assert(false)
+ } else {
+ assert(true)
+ }
+ }
+
+ override def afterAll: Unit = {
+ sql("drop table if exists mainTable")
+ sql("drop table if exists agg0")
+ sql("drop table if exists agg1")
+ sql("drop table if exists agg2")
+ sql("drop table if exists agg3")
+ sql("drop table if exists agg4")
+ sql("drop table if exists agg5")
+ sql("drop table if exists agg6")
+ sql("drop table if exists agg7")
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index 42447da..e83d96a 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -173,6 +173,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
protected val DATAMAP = carbonKeyWord("DATAMAP")
protected val ON = carbonKeyWord("ON")
protected val DMPROPERTIES = carbonKeyWord("DMPROPERTIES")
+ protected val SELECT = carbonKeyWord("SELECT")
protected val doubleQuotedString = "\"([^\"]+)\"".r
protected val singleQuotedString = "'([^']+)'".r
@@ -989,6 +990,12 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
Field(e1, e2.dataType, Some(e1), e2.children, null, e3)
}
+ lazy val addPreAgg: Parser[String] =
+ SELECT ~> restInput <~ opt(";") ^^ {
+ case query =>
+ "select preAGG() as preAgg, " + query
+ }
+
protected lazy val primitiveFieldType: Parser[Field] =
primitiveTypes ^^ {
case e1 =>
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 9899be1..28dcbf2 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -463,7 +463,7 @@ object CarbonDataRDDFactory {
throw new Exception(status(0)._2._2.errorMsg)
}
// if segment is empty then fail the data load
- if (!carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isPreAggregateTable &&
+ if (!carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isChildDataMap &&
!CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
// update the load entry in table status file for changing the status to failure
CommonUtil.updateTableStatusForFailure(carbonLoadModel)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
index a37b55b..b69ef2f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
@@ -52,6 +52,9 @@ class CarbonEnv {
def init(sparkSession: SparkSession): Unit = {
sparkSession.udf.register("getTupleId", () => "")
+ // added for handling preaggregate table creation. when user will fire create ddl for
+ // create table we are adding a udf so no need to apply PreAggregate rules.
+ sparkSession.udf.register("preAgg", () => "")
if (!initialized) {
// update carbon session parameters , preserve thread parameters
val currentThreadSesssionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index ebf6273..3a78968 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -25,6 +25,8 @@ import org.apache.spark.sql.execution.command._
import org.apache.spark.sql.hive.CarbonRelation
import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
/**
* Below command class will be used to create pre-aggregate table
* and updating the parent table about the child table information
@@ -47,9 +49,10 @@ case class CreatePreAggregateTableCommand(
}
override def processSchema(sparkSession: SparkSession): Seq[Row] = {
- val df = sparkSession.sql(queryString)
- val fieldRelationMap = PreAggregateUtil
- .validateActualSelectPlanAndGetAttrubites(df.logicalPlan, queryString)
+ val updatedQuery = new CarbonSpark2SqlParser().addPreAggFunction(queryString)
+ val df = sparkSession.sql(updatedQuery)
+ val fieldRelationMap = PreAggregateUtil.validateActualSelectPlanAndGetAttributes(
+ df.logicalPlan, queryString)
val fields = fieldRelationMap.keySet.toSeq
val tableProperties = mutable.Map[String, String]()
dmproperties.foreach(t => tableProperties.put(t._1, t._2))
@@ -87,7 +90,8 @@ case class CreatePreAggregateTableCommand(
val tableInfo = relation.tableMeta.carbonTable.getTableInfo
// child schema object which will be updated on parent table about the
val childSchema = tableInfo.getFactTable.buildChildSchema(
- dataMapName, "", tableInfo.getDatabaseName, queryString, "AGGREGATION")
+ dataMapName, CarbonCommonConstants.AGGREGATIONDATAMAPSCHEMA,
+ tableInfo.getDatabaseName, queryString, "AGGREGATION")
dmproperties.foreach(f => childSchema.getProperties.put(f._1, f._2))
// updating the parent table about child table
PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
index 8271e57..7a66e88 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -105,7 +105,7 @@ object PreAggregateDataTypeChangePreListener extends OperationEventListener {
}
}
- if (carbonTable.isPreAggregateTable) {
+ if (carbonTable.isChildDataMap) {
throw new UnsupportedOperationException(s"Cannot change data type for columns in " +
s"pre-aggreagate table ${
carbonTable.getDatabaseName
@@ -126,12 +126,12 @@ object PreAggregateDeleteSegmentByDatePreListener extends OperationEventListener
val deleteSegmentByDatePreEvent = event.asInstanceOf[DeleteSegmentByDatePreEvent]
val carbonTable = deleteSegmentByDatePreEvent.carbonTable
if (carbonTable != null) {
- if (carbonTable.hasPreAggregateTables) {
+ if (carbonTable.hasDataMapSchema) {
throw new UnsupportedOperationException(
"Delete segment operation is not supported on tables which have a pre-aggregate table. " +
"Drop pre-aggregation table to continue")
}
- if (carbonTable.isPreAggregateTable) {
+ if (carbonTable.isChildDataMap) {
throw new UnsupportedOperationException(
"Delete segment operation is not supported on pre-aggregate table")
}
@@ -150,11 +150,11 @@ object PreAggregateDeleteSegmentByIdPreListener extends OperationEventListener {
val tableEvent = event.asInstanceOf[DeleteSegmentByIdPreEvent]
val carbonTable = tableEvent.carbonTable
if (carbonTable != null) {
- if (carbonTable.hasPreAggregateTables) {
+ if (carbonTable.hasDataMapSchema) {
throw new UnsupportedOperationException(
"Delete segment operation is not supported on tables which have a pre-aggregate table")
}
- if (carbonTable.isPreAggregateTable) {
+ if (carbonTable.isChildDataMap) {
throw new UnsupportedOperationException(
"Delete segment operation is not supported on pre-aggregate table")
}
@@ -190,7 +190,7 @@ object PreAggregateDropColumnPreListener extends OperationEventListener {
s"pre-aggregate table ${ dataMapSchema.getRelationIdentifier.toString}")
}
}
- if (carbonTable.isPreAggregateTable) {
+ if (carbonTable.isChildDataMap) {
throw new UnsupportedOperationException(s"Cannot drop columns in pre-aggreagate table ${
carbonTable.getDatabaseName}.${ carbonTable.getFactTableName }")
}
@@ -209,11 +209,11 @@ object PreAggregateRenameTablePreListener extends OperationEventListener {
operationContext: OperationContext): Unit = {
val renameTablePostListener = event.asInstanceOf[AlterTableRenamePreEvent]
val carbonTable = renameTablePostListener.carbonTable
- if (carbonTable.isPreAggregateTable) {
+ if (carbonTable.isChildDataMap) {
throw new UnsupportedOperationException(
"Rename operation for pre-aggregate table is not supported.")
}
- if (carbonTable.hasPreAggregateTables) {
+ if (carbonTable.hasDataMapSchema) {
throw new UnsupportedOperationException(
"Rename operation is not supported for table with pre-aggregate tables")
}
@@ -231,12 +231,12 @@ object UpdatePreAggregatePreListener extends OperationEventListener {
val tableEvent = event.asInstanceOf[UpdateTablePreEvent]
val carbonTable = tableEvent.carbonTable
if (carbonTable != null) {
- if (carbonTable.hasPreAggregateTables) {
+ if (carbonTable.hasDataMapSchema) {
throw new UnsupportedOperationException(
"Update operation is not supported for tables which have a pre-aggregate table. Drop " +
"pre-aggregate tables to continue.")
}
- if (carbonTable.isPreAggregateTable) {
+ if (carbonTable.isChildDataMap) {
throw new UnsupportedOperationException(
"Update operation is not supported for pre-aggregate table")
}
@@ -255,12 +255,12 @@ object DeletePreAggregatePreListener extends OperationEventListener {
val tableEvent = event.asInstanceOf[DeleteFromTablePreEvent]
val carbonTable = tableEvent.carbonTable
if (carbonTable != null) {
- if (carbonTable.hasPreAggregateTables) {
+ if (carbonTable.hasDataMapSchema) {
throw new UnsupportedOperationException(
"Delete operation is not supported for tables which have a pre-aggregate table. Drop " +
"pre-aggregate tables to continue.")
}
- if (carbonTable.isPreAggregateTable) {
+ if (carbonTable.isChildDataMap) {
throw new UnsupportedOperationException(
"Delete operation is not supported for pre-aggregate table")
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index b926705..62e7623 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -16,13 +16,14 @@
*/
package org.apache.spark.sql.execution.command.preaaggregate
-import scala.collection.mutable.ListBuffer
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import scala.collection.JavaConverters._
import org.apache.spark.SparkConf
import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast}
+import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedFunction, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, NamedExpression, ScalaUDF}
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.command.{ColumnTableRelation, DataMapField, Field}
@@ -51,9 +52,14 @@ object PreAggregateUtil {
def getParentCarbonTable(plan: LogicalPlan): CarbonTable = {
plan match {
- case Aggregate(_, aExp, SubqueryAlias(_, l: LogicalRelation, _))
- if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
- l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation.metaData.carbonTable
+ case Aggregate(_, _, SubqueryAlias(_, logicalRelation: LogicalRelation, _))
+ if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+ logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].
+ carbonRelation.metaData.carbonTable
+ case Aggregate(_, _, logicalRelation: LogicalRelation)
+ if logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+ logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].
+ carbonRelation.metaData.carbonTable
case _ => throw new MalformedCarbonCommandException("table does not exist")
}
}
@@ -67,54 +73,86 @@ object PreAggregateUtil {
* @param selectStmt
* @return list of fields
*/
- def validateActualSelectPlanAndGetAttrubites(plan: LogicalPlan,
+ def validateActualSelectPlanAndGetAttributes(plan: LogicalPlan,
selectStmt: String): scala.collection.mutable.LinkedHashMap[Field, DataMapField] = {
- val fieldToDataMapFieldMap = scala.collection.mutable.LinkedHashMap.empty[Field, DataMapField]
plan match {
- case Aggregate(_, aExp, SubqueryAlias(_, l: LogicalRelation, _))
- if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
- val carbonTable = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation
- .metaData.carbonTable
- val parentTableName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
- .getTableName
- val parentDatabaseName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
- .getDatabaseName
- val parentTableId = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
- .getTableId
- if (!carbonTable.getTableInfo.getParentRelationIdentifiers.isEmpty) {
+ case Aggregate(groupByExp, aggExp, SubqueryAlias(_, logicalRelation: LogicalRelation, _)) =>
+ getFieldsFromPlan(groupByExp, aggExp, logicalRelation, selectStmt)
+ case Aggregate(groupByExp, aggExp, logicalRelation: LogicalRelation) =>
+ getFieldsFromPlan(groupByExp, aggExp, logicalRelation, selectStmt)
+ }
+ }
+
+ /**
+ * Below method will be used to get the fields from expressions
+ * @param groupByExp
+ * grouping expression
+ * @param aggExp
+ * aggregate expression
+ * @param logicalRelation
+ * logical relation
+ * @param selectStmt
+ * select statement
+ * @return fields from expressions
+ */
+ def getFieldsFromPlan(groupByExp: Seq[Expression],
+ aggExp: Seq[NamedExpression], logicalRelation: LogicalRelation, selectStmt: String):
+ scala.collection.mutable.LinkedHashMap[Field, DataMapField] = {
+ val fieldToDataMapFieldMap = scala.collection.mutable.LinkedHashMap.empty[Field, DataMapField]
+ if (!logicalRelation.relation.isInstanceOf[CarbonDatasourceHadoopRelation]) {
+ throw new MalformedCarbonCommandException("Un-supported table")
+ }
+ val carbonTable = logicalRelation.relation.
+ asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation
+ .metaData.carbonTable
+ val parentTableName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+ .getTableName
+ val parentDatabaseName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+ .getDatabaseName
+ val parentTableId = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+ .getTableId
+ if (!carbonTable.getTableInfo.getParentRelationIdentifiers.isEmpty) {
+ throw new MalformedCarbonCommandException(
+ "Pre Aggregation is not supported on Pre-Aggregated Table")
+ }
+ groupByExp.map {
+ case attr: AttributeReference =>
+ fieldToDataMapFieldMap += getField(attr.name,
+ attr.dataType,
+ parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName = parentTableName,
+ parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+ case _ =>
+ throw new MalformedCarbonCommandException(s"Unsupported Function in select Statement:${
+ selectStmt } ")
+ }
+ aggExp.map {
+ case Alias(attr: AggregateExpression, _) =>
+ if (attr.isDistinct) {
throw new MalformedCarbonCommandException(
- "Pre Aggregation is not supported on Pre-Aggregated Table")
- }
- aExp.map {
- case Alias(attr: AggregateExpression, _) =>
- if (attr.isDistinct) {
- throw new MalformedCarbonCommandException(
- "Distinct is not supported On Pre Aggregation")
- }
- fieldToDataMapFieldMap ++= (validateAggregateFunctionAndGetFields(carbonTable,
- attr.aggregateFunction,
- parentTableName,
- parentDatabaseName,
- parentTableId))
- case attr: AttributeReference =>
- fieldToDataMapFieldMap += getField(attr.name,
- attr.dataType,
- parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
- parentTableName = parentTableName,
- parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
- case Alias(attr: AttributeReference, _) =>
- fieldToDataMapFieldMap += getField(attr.name,
- attr.dataType,
- parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
- parentTableName = parentTableName,
- parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
- case _ =>
- throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${
- selectStmt } ")
+ "Distinct is not supported On Pre Aggregation")
}
- Some(carbonTable)
+ fieldToDataMapFieldMap ++= validateAggregateFunctionAndGetFields(carbonTable,
+ attr.aggregateFunction,
+ parentTableName,
+ parentDatabaseName,
+ parentTableId)
+ case attr: AttributeReference =>
+ fieldToDataMapFieldMap += getField(attr.name,
+ attr.dataType,
+ parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName = parentTableName,
+ parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+ case Alias(attr: AttributeReference, _) =>
+ fieldToDataMapFieldMap += getField(attr.name,
+ attr.dataType,
+ parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName = parentTableName,
+ parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+ case _@Alias(s: ScalaUDF, name) if name.equals("preAgg") =>
case _ =>
- throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${ selectStmt } ")
+ throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${
+ selectStmt } ")
}
fieldToDataMapFieldMap
}
@@ -347,30 +385,6 @@ object PreAggregateUtil {
}
/**
- * This method will split schema string into multiple parts of configured size and
- * registers the parts as keys in tableProperties which will be read by spark to prepare
- * Carbon Table fields
- *
- * @param sparkConf
- * @param schemaJsonString
- * @return
- */
- private def prepareSchemaJson(sparkConf: SparkConf,
- schemaJsonString: String): String = {
- val threshold = sparkConf
- .getInt(CarbonCommonConstants.SPARK_SCHEMA_STRING_LENGTH_THRESHOLD,
- CarbonCommonConstants.SPARK_SCHEMA_STRING_LENGTH_THRESHOLD_DEFAULT)
- // Split the JSON string.
- val parts = schemaJsonString.grouped(threshold).toSeq
- var schemaParts: Seq[String] = Seq.empty
- schemaParts = schemaParts :+ s"'$DATASOURCE_SCHEMA_NUMPARTS'='${ parts.size }'"
- parts.zipWithIndex.foreach { case (part, index) =>
- schemaParts = schemaParts :+ s"'$DATASOURCE_SCHEMA_PART_PREFIX$index'='$part'"
- }
- schemaParts.mkString(",")
- }
-
- /**
* Validates that the table exists and acquires meta lock on it.
*
* @param dbName
@@ -453,4 +467,30 @@ object PreAggregateUtil {
def checkMainTableLoad(carbonTable: CarbonTable): Boolean = {
SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath).nonEmpty
}
+
+ /**
+ * Below method will be used to update logical plan
+ * this is required for creating pre aggregate tables,
+ * so @CarbonPreAggregateRules will not be applied during creation
+ * @param logicalPlan
+ * actual logical plan
+ * @return updated plan
+ */
+ def updatePreAggQueyPlan(logicalPlan: LogicalPlan): LogicalPlan = {
+ val updatedPlan = logicalPlan.transform {
+ case _@Project(projectList, child) =>
+ val buffer = new ArrayBuffer[NamedExpression]()
+ buffer ++= projectList
+ buffer += UnresolvedAlias(Alias(UnresolvedFunction("preAgg",
+ Seq.empty, isDistinct = false), "preAgg")())
+ Project(buffer, child)
+ case Aggregate(groupByExp, aggExp, l: UnresolvedRelation) =>
+ val buffer = new ArrayBuffer[NamedExpression]()
+ buffer ++= aggExp
+ buffer += UnresolvedAlias(Alias(UnresolvedFunction("preAgg",
+ Seq.empty, isDistinct = false), "preAgg")())
+ Aggregate(groupByExp, buffer, l)
+ }
+ updatedPlan
+ }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e4f2843c/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 ba7e1eb..7bd0fad 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,111 +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, AttributeReference, Cast, ExprId, NamedExpression}
-import org.apache.spark.sql.catalyst.expressions.aggregate.{DeclarativeAggregate, _}
+import org.apache.spark.sql.catalyst.expressions.Alias
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.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
-
-
-/**
- * Insert into carbon table from other source
- */
-object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
- def apply(plan: LogicalPlan): LogicalPlan = {
- plan.transform {
- // Wait until children are resolved.
- case p: LogicalPlan if !p.childrenResolved => p
-
- case p@InsertIntoTable(relation: LogicalRelation, _, child, _, _)
- if relation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
- castChildOutput(p, relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation], child)
- }
- }
-
- def castChildOutput(p: InsertIntoTable,
- relation: CarbonDatasourceHadoopRelation,
- child: LogicalPlan)
- : LogicalPlan = {
- if (relation.carbonRelation.output.size > CarbonCommonConstants
- .DEFAULT_MAX_NUMBER_OF_COLUMNS) {
- sys
- .error("Maximum supported column by carbon is:" + CarbonCommonConstants
- .DEFAULT_MAX_NUMBER_OF_COLUMNS
- )
- }
- val isAggregateTable = !relation.carbonRelation.tableMeta.carbonTable.getTableInfo
- .getParentRelationIdentifiers.isEmpty
- // transform logical plan if the load is for aggregate table.
- val childPlan = if (isAggregateTable) {
- transformAggregatePlan(child)
- } else {
- child
- }
- if (childPlan.output.size >= relation.carbonRelation.output.size) {
- val newChildOutput = childPlan.output.zipWithIndex.map { columnWithIndex =>
- columnWithIndex._1 match {
- case attr: Alias =>
- Alias(attr.child, s"col${ columnWithIndex._2 }")(attr.exprId)
- case attr: Attribute =>
- Alias(attr, s"col${ columnWithIndex._2 }")(NamedExpression.newExprId)
- case attr => attr
- }
- }
- val newChild: LogicalPlan = if (newChildOutput == childPlan.output) {
- p.child
- } else {
- Project(newChildOutput, childPlan)
- }
- InsertIntoCarbonTable(relation, p.partition, newChild, p.overwrite, p.ifNotExists)
- } else {
- sys.error("Cannot insert into target table because column number are different")
- }
- }
-
- /**
- * Transform the logical plan with average(col1) aggregation type to sum(col1) and count(col1).
- *
- * @param logicalPlan
- * @return
- */
- private def transformAggregatePlan(logicalPlan: LogicalPlan): LogicalPlan = {
- logicalPlan transform {
- case aggregate@Aggregate(_, aExp, _) =>
- val newExpressions = aExp flatMap {
- case alias@Alias(attrExpression: AggregateExpression, _) =>
- attrExpression.aggregateFunction flatMap {
- case Average(attr: AttributeReference) =>
- Seq(Alias(attrExpression
- .copy(aggregateFunction = Sum(attr.withName(attr.name)),
- resultId = NamedExpression.newExprId),
- attr.name)(),
- Alias(attrExpression
- .copy(aggregateFunction = Count(attr.withName(attr.name)),
- resultId = NamedExpression.newExprId), attr.name)())
- case Average(Cast(attr: AttributeReference, _)) =>
- Seq(Alias(attrExpression
- .copy(aggregateFunction = Sum(attr.withName(attr.name)),
- resultId = NamedExpression.newExprId),
- attr.name)(),
- Alias(attrExpression
- .copy(aggregateFunction = Count(attr.withName(attr.name)),
- resultId = NamedExpression.newExprId), attr.name)())
- case _: DeclarativeAggregate => Seq(alias)
- case _ => Nil
- }
- case namedExpr: NamedExpression => Seq(namedExpr)
- }
- aggregate.copy(aggregateExpressions = newExpressions)
- case plan: LogicalPlan => plan
- }
- }
-}
case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[LogicalPlan] {
[5/9] carbondata git commit: [CARBONDATA-1609] Thrift Changes to
support Pre-aggregate
Posted by ra...@apache.org.
[CARBONDATA-1609] Thrift Changes to support Pre-aggregate
This closes #1425
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/31699185
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/31699185
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/31699185
Branch: refs/heads/master
Commit: 3169918523a42c8f2c1d612052a7680514372bf9
Parents: 40c31e8
Author: kumarvishal <ku...@gmail.com>
Authored: Sat Oct 21 16:31:15 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530
----------------------------------------------------------------------
.../metadata/converter/SchemaConverter.java | 10 ++
.../ThriftWrapperSchemaConverterImpl.java | 147 +++++++++++++++++--
.../core/metadata/schema/table/CarbonTable.java | 24 +--
.../metadata/schema/table/DataMapSchema.java | 121 +++++++++++++++
.../schema/table/RelationIdentifier.java | 88 +++++++++++
.../core/metadata/schema/table/TableInfo.java | 83 ++++++++++-
.../core/metadata/schema/table/TableSchema.java | 26 ++++
.../schema/table/column/ColumnSchema.java | 48 +++++-
.../util/AbstractDataFileFooterConverter.java | 28 ++++
format/src/main/thrift/schema.thrift | 31 ++++
10 files changed, 580 insertions(+), 26 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
index 4faa017..bfbb6f7 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.core.metadata.converter;
import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
import org.apache.carbondata.core.metadata.schema.table.TableInfo;
import org.apache.carbondata.core.metadata.schema.table.TableSchema;
import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -100,4 +101,13 @@ public interface SchemaConverter {
*/
TableInfo fromExternalToWrapperTableInfo(org.apache.carbondata.format.TableInfo externalTableInfo,
String dbName, String tableName, String storePath);
+
+ /**
+ * method to convert thrift datamap schema object to wrapper
+ * data map object
+ * @param thriftchildSchema
+ * @return DataMapSchema
+ */
+ DataMapSchema fromExternalToWrapperDataMapSchema(
+ org.apache.carbondata.format.DataMapSchema thriftchildSchema);
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index adcac7d..5ffc612 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -30,9 +30,12 @@ import org.apache.carbondata.core.metadata.schema.PartitionInfo;
import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
import org.apache.carbondata.core.metadata.schema.table.TableInfo;
import org.apache.carbondata.core.metadata.schema.table.TableSchema;
import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
/**
* Thrift schema to carbon schema converter and vice versa
@@ -40,11 +43,11 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
/* (non-Javadoc)
- * Converts from wrapper to thrift schema evolution entry
- */
+ * Converts from wrapper to thrift schema evolution entry
+ */
@Override
public org.apache.carbondata.format.SchemaEvolutionEntry
- fromWrapperToExternalSchemaEvolutionEntry(SchemaEvolutionEntry wrapperSchemaEvolutionEntry) {
+ fromWrapperToExternalSchemaEvolutionEntry(SchemaEvolutionEntry wrapperSchemaEvolutionEntry) {
org.apache.carbondata.format.SchemaEvolutionEntry thriftSchemaEvolutionEntry =
new org.apache.carbondata.format.SchemaEvolutionEntry(
wrapperSchemaEvolutionEntry.getTimeStamp());
@@ -159,7 +162,6 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
*/
@Override public org.apache.carbondata.format.ColumnSchema fromWrapperToExternalColumnSchema(
ColumnSchema wrapperColumnSchema) {
-
List<org.apache.carbondata.format.Encoding> encoders =
new ArrayList<org.apache.carbondata.format.Encoding>();
for (Encoding encoder : wrapperColumnSchema.getEncodingList()) {
@@ -188,13 +190,18 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
thriftColumnSchema.setInvisible(wrapperColumnSchema.isInvisible());
thriftColumnSchema.setColumnReferenceId(wrapperColumnSchema.getColumnReferenceId());
thriftColumnSchema.setSchemaOrdinal(wrapperColumnSchema.getSchemaOrdinal());
-
if (wrapperColumnSchema.isSortColumn()) {
Map<String, String> properties = new HashMap<String, String>();
properties.put(CarbonCommonConstants.SORT_COLUMNS, "true");
thriftColumnSchema.setColumnProperties(properties);
}
-
+ thriftColumnSchema.setAggregate_function(wrapperColumnSchema.getAggFunction());
+ List<ParentColumnTableRelation> parentColumnTableRelations =
+ wrapperColumnSchema.getParentColumnTableRelations();
+ if (null != parentColumnTableRelations) {
+ thriftColumnSchema.setParentColumnTableRelations(
+ wrapperToThriftRelationList(parentColumnTableRelations));
+ }
return thriftColumnSchema;
}
@@ -280,11 +287,79 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
*/
@Override public org.apache.carbondata.format.TableInfo fromWrapperToExternalTableInfo(
TableInfo wrapperTableInfo, String dbName, String tableName) {
-
org.apache.carbondata.format.TableSchema thriftFactTable =
fromWrapperToExternalTableSchema(wrapperTableInfo.getFactTable());
- return new org.apache.carbondata.format.TableInfo(thriftFactTable, new ArrayList<org.apache
- .carbondata.format.TableSchema>());
+ org.apache.carbondata.format.TableInfo tableInfo =
+ new org.apache.carbondata.format.TableInfo(thriftFactTable,
+ new ArrayList<org.apache.carbondata.format.TableSchema>());
+ List<DataMapSchema> wrapperChildSchemaList = wrapperTableInfo.getDataMapSchemaList();
+ if (null != wrapperChildSchemaList) {
+ List<org.apache.carbondata.format.DataMapSchema> thriftChildSchemas =
+ fromWrapperToExternalChildSchemaList(wrapperChildSchemaList);
+ tableInfo.setDataMapSchemas(thriftChildSchemas);
+ }
+ return tableInfo;
+ }
+
+ private List<org.apache.carbondata.format.RelationIdentifier> fromWrapperToExternalRI(
+ List<RelationIdentifier> relationIdentifiersList) {
+ List<org.apache.carbondata.format.RelationIdentifier> thriftRelationIdentifierList =
+ new ArrayList<>();
+ for (RelationIdentifier relationIdentifier : relationIdentifiersList) {
+ org.apache.carbondata.format.RelationIdentifier thriftRelationIdentifier =
+ new org.apache.carbondata.format.RelationIdentifier();
+ thriftRelationIdentifier.setDatabaseName(relationIdentifier.getDatabaseName());
+ thriftRelationIdentifier.setTableName(relationIdentifier.getTableName());
+ thriftRelationIdentifier.setTableId(relationIdentifier.getTableId());
+ thriftRelationIdentifierList.add(thriftRelationIdentifier);
+ }
+ return thriftRelationIdentifierList;
+ }
+
+ private List<org.apache.carbondata.format.DataMapSchema> fromWrapperToExternalChildSchemaList(
+ List<DataMapSchema> wrapperChildSchemaList) {
+ List<org.apache.carbondata.format.DataMapSchema> thriftChildSchemas = new ArrayList<>();
+ for (DataMapSchema wrapperChildSchema : wrapperChildSchemaList) {
+ org.apache.carbondata.format.DataMapSchema thriftChildSchema =
+ new org.apache.carbondata.format.DataMapSchema();
+ org.apache.carbondata.format.RelationIdentifier relationIdentifier =
+ new org.apache.carbondata.format.RelationIdentifier();
+ relationIdentifier
+ .setDatabaseName(wrapperChildSchema.getRelationIdentifier().getDatabaseName());
+ relationIdentifier.setTableName(wrapperChildSchema.getRelationIdentifier().getTableName());
+ relationIdentifier.setTableId(wrapperChildSchema.getRelationIdentifier().getTableId());
+ thriftChildSchema.setRelationIdentifire(relationIdentifier);
+ thriftChildSchema.setProperties(wrapperChildSchema.getProperties());
+ thriftChildSchema.setClassName(wrapperChildSchema.getClassName());
+ thriftChildSchema.setChildTableSchema(
+ fromWrapperToExternalTableSchema(wrapperChildSchema.getChildSchema()));
+ thriftChildSchemas.add(thriftChildSchema);
+ }
+ return thriftChildSchemas;
+ }
+
+ private List<org.apache.carbondata.format.ParentColumnTableRelation> wrapperToThriftRelationList(
+ List<ParentColumnTableRelation> wrapperColumnRelations) {
+ List<org.apache.carbondata.format.ParentColumnTableRelation> thriftColumnRelationList =
+ new ArrayList<>();
+
+ for (ParentColumnTableRelation wrapperColumnRealtion : wrapperColumnRelations) {
+ org.apache.carbondata.format.ParentColumnTableRelation thriftColumnTableRelation =
+ new org.apache.carbondata.format.ParentColumnTableRelation();
+ thriftColumnTableRelation.setColumnId(wrapperColumnRealtion.getColumnId());
+ thriftColumnTableRelation.setColumnName(wrapperColumnRealtion.getColumnName());
+ org.apache.carbondata.format.RelationIdentifier thriftRelationIdentifier =
+ new org.apache.carbondata.format.RelationIdentifier();
+ thriftRelationIdentifier
+ .setDatabaseName(wrapperColumnRealtion.getRelationIdentifier().getDatabaseName());
+ thriftRelationIdentifier
+ .setTableName(wrapperColumnRealtion.getRelationIdentifier().getTableName());
+ thriftRelationIdentifier
+ .setTableId(wrapperColumnRealtion.getRelationIdentifier().getTableId());
+ thriftColumnTableRelation.setRelationIdentifier(thriftRelationIdentifier);
+ thriftColumnRelationList.add(thriftColumnTableRelation);
+ }
+ return thriftColumnRelationList;
}
/* (non-Javadoc)
@@ -436,9 +511,17 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
wrapperColumnSchema.setSortColumn(true);
}
}
+ wrapperColumnSchema.setAggFunction(externalColumnSchema.getAggregate_function());
+ List<org.apache.carbondata.format.ParentColumnTableRelation> parentColumnTableRelation =
+ externalColumnSchema.getParentColumnTableRelations();
+ if (null != parentColumnTableRelation) {
+ wrapperColumnSchema.setParentColumnTableRelations(
+ fromExtrenalToWrapperParentTableColumnRelations(parentColumnTableRelation));
+ }
return wrapperColumnSchema;
}
+
private PartitionType fromExternalToWrapperPartitionType(
org.apache.carbondata.format.PartitionType externalPartitionType) {
if (null == externalPartitionType) {
@@ -508,7 +591,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
org.apache.carbondata.format.BucketingInfo externalBucketInfo) {
List<ColumnSchema> listOfColumns = new ArrayList<ColumnSchema>();
for (org.apache.carbondata.format.ColumnSchema externalColumnSchema :
- externalBucketInfo.table_columns) {
+ externalBucketInfo.table_columns) {
listOfColumns.add(fromExternalToWrapperColumnSchema(externalColumnSchema));
}
return new BucketingInfo(listOfColumns, externalBucketInfo.number_of_buckets);
@@ -531,7 +614,51 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
wrapperTableInfo.setStorePath(storePath);
wrapperTableInfo.setFactTable(
fromExternalToWrapperTableSchema(externalTableInfo.getFact_table(), tableName));
+ if (null != externalTableInfo.getDataMapSchemas()) {
+ wrapperTableInfo.setDataMapSchemaList(
+ fromExternalToWrapperChildSchemaList(externalTableInfo.getDataMapSchemas()));
+ }
return wrapperTableInfo;
}
+ @Override public DataMapSchema fromExternalToWrapperDataMapSchema(
+ org.apache.carbondata.format.DataMapSchema thriftDataMapSchema) {
+ RelationIdentifier relationIdentifier =
+ new RelationIdentifier(thriftDataMapSchema.getRelationIdentifire().getDatabaseName(),
+ thriftDataMapSchema.getRelationIdentifire().getTableName(),
+ thriftDataMapSchema.getRelationIdentifire().getTableId());
+ DataMapSchema childSchema = new DataMapSchema(thriftDataMapSchema.getClassName());
+ childSchema.setProperties(thriftDataMapSchema.getProperties());
+ childSchema.setChildSchema(
+ fromExternalToWrapperTableSchema(thriftDataMapSchema.getChildTableSchema(),
+ relationIdentifier.getTableName()));
+ childSchema.setRelationIdentifier(relationIdentifier);
+ return childSchema;
+ }
+
+ private List<ParentColumnTableRelation> fromExtrenalToWrapperParentTableColumnRelations(
+ List<org.apache.carbondata.format.ParentColumnTableRelation> thirftParentColumnRelation) {
+ List<ParentColumnTableRelation> parentColumnTableRelationList = new ArrayList<>();
+ for (org.apache.carbondata.format.ParentColumnTableRelation carbonTableRelation :
+ thirftParentColumnRelation) {
+ RelationIdentifier relationIdentifier =
+ new RelationIdentifier(carbonTableRelation.getRelationIdentifier().getDatabaseName(),
+ carbonTableRelation.getRelationIdentifier().getTableName(),
+ carbonTableRelation.getRelationIdentifier().getTableId());
+ ParentColumnTableRelation parentColumnTableRelation =
+ new ParentColumnTableRelation(relationIdentifier, carbonTableRelation.getColumnId(),
+ carbonTableRelation.getColumnName());
+ parentColumnTableRelationList.add(parentColumnTableRelation);
+ }
+ return parentColumnTableRelationList;
+ }
+
+ public List<DataMapSchema> fromExternalToWrapperChildSchemaList(
+ List<org.apache.carbondata.format.DataMapSchema> childSchemaList) {
+ List<DataMapSchema> childSchemas = new ArrayList<>();
+ for (org.apache.carbondata.format.DataMapSchema childSchemaThrift : childSchemaList) {
+ childSchemas.add(fromExternalToWrapperDataMapSchema(childSchemaThrift));
+ }
+ return childSchemas;
+ }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index d4aaa29..e63f4e3 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -214,7 +214,7 @@ public class CarbonTable implements Serializable {
if (columnSchema.getNumberOfChild() > 0) {
CarbonDimension complexDimension =
new CarbonDimension(columnSchema, dimensionOrdinal++,
- columnSchema.getSchemaOrdinal(), -1, -1, ++complexTypeOrdinal);
+ columnSchema.getSchemaOrdinal(), -1, -1, ++complexTypeOrdinal);
complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
allDimensions.add(complexDimension);
dimensionOrdinal =
@@ -228,8 +228,8 @@ public class CarbonTable implements Serializable {
}
if (!columnSchema.getEncodingList().contains(Encoding.DICTIONARY)) {
CarbonDimension dimension =
- new CarbonDimension(columnSchema, dimensionOrdinal++,
- columnSchema.getSchemaOrdinal(), -1, -1, -1);
+ new CarbonDimension(columnSchema, dimensionOrdinal++,
+ columnSchema.getSchemaOrdinal(), -1, -1, -1);
if (!columnSchema.isInvisible() && columnSchema.isSortColumn()) {
this.numberOfNoDictSortColumns++;
}
@@ -238,8 +238,8 @@ public class CarbonTable implements Serializable {
} else if (columnSchema.getEncodingList().contains(Encoding.DICTIONARY)
&& columnSchema.getColumnGroupId() == -1) {
CarbonDimension dimension =
- new CarbonDimension(columnSchema, dimensionOrdinal++,
- columnSchema.getSchemaOrdinal(), keyOrdinal++, -1, -1);
+ new CarbonDimension(columnSchema, dimensionOrdinal++,
+ columnSchema.getSchemaOrdinal(), keyOrdinal++, -1, -1);
allDimensions.add(dimension);
primitiveDimensions.add(dimension);
} else {
@@ -247,15 +247,15 @@ public class CarbonTable implements Serializable {
previousColumnGroupId == columnSchema.getColumnGroupId() ? ++columnGroupOrdinal : 0;
previousColumnGroupId = columnSchema.getColumnGroupId();
CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++,
- columnSchema.getSchemaOrdinal(), keyOrdinal++,
- columnGroupOrdinal, -1);
+ columnSchema.getSchemaOrdinal(), keyOrdinal++,
+ columnGroupOrdinal, -1);
allDimensions.add(dimension);
primitiveDimensions.add(dimension);
}
}
} else {
allMeasures.add(new CarbonMeasure(columnSchema, measureOrdinal++,
- columnSchema.getSchemaOrdinal()));
+ columnSchema.getSchemaOrdinal()));
}
}
fillVisibleDimensions(tableSchema.getTableName());
@@ -290,14 +290,14 @@ public class CarbonTable implements Serializable {
*/
private int readAllComplexTypeChildrens(int dimensionOrdinal, int childCount,
List<ColumnSchema> listOfColumns, CarbonDimension parentDimension,
- List<CarbonDimension> primitiveDimensions) {
+ List<CarbonDimension> primitiveDimensions) {
for (int i = 0; i < childCount; i++) {
ColumnSchema columnSchema = listOfColumns.get(dimensionOrdinal);
if (columnSchema.isDimensionColumn()) {
if (columnSchema.getNumberOfChild() > 0) {
CarbonDimension complexDimension =
new CarbonDimension(columnSchema, dimensionOrdinal++,
- columnSchema.getSchemaOrdinal(), -1, -1, -1);
+ columnSchema.getSchemaOrdinal(), -1, -1, -1);
complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
parentDimension.getListOfChildDimensions().add(complexDimension);
dimensionOrdinal =
@@ -305,8 +305,8 @@ public class CarbonTable implements Serializable {
listOfColumns, complexDimension, primitiveDimensions);
} else {
CarbonDimension carbonDimension =
- new CarbonDimension(columnSchema, dimensionOrdinal++,
- columnSchema.getSchemaOrdinal(), -1, -1, -1);
+ new CarbonDimension(columnSchema, dimensionOrdinal++,
+ columnSchema.getSchemaOrdinal(), -1, -1, -1);
parentDimension.getListOfChildDimensions().add(carbonDimension);
primitiveDimensions.add(carbonDimension);
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
new file mode 100644
index 0000000..5f92ec8
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.metadata.schema.table;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Child schema class to maintain the child table details inside parent table
+ */
+public class DataMapSchema implements Serializable, Writable {
+
+ private String className;
+
+ private RelationIdentifier relationIdentifier;
+ /**
+ * child table schema
+ */
+ private TableSchema childSchema;
+
+ /**
+ * relation properties
+ */
+ private Map<String, String> properties;
+
+ public DataMapSchema(String className) {
+ this.className = className;
+ }
+
+ public String getClassName() {
+ return className;
+ }
+
+ public TableSchema getChildSchema() {
+ return childSchema;
+ }
+
+ public RelationIdentifier getRelationIdentifier() {
+ return relationIdentifier;
+ }
+
+ public Map<String, String> getProperties() {
+ return properties;
+ }
+
+ public void setRelationIdentifier(RelationIdentifier relationIdentifier) {
+ this.relationIdentifier = relationIdentifier;
+ }
+
+ public void setChildSchema(TableSchema childSchema) {
+ this.childSchema = childSchema;
+ }
+
+ public void setProperties(Map<String, String> properties) {
+ this.properties = properties;
+ }
+
+ @Override public void write(DataOutput out) throws IOException {
+ out.writeUTF(className);
+ boolean isRelationIdentifierExists = null != relationIdentifier;
+ out.writeBoolean(isRelationIdentifierExists);
+ if (isRelationIdentifierExists) {
+ this.relationIdentifier.write(out);
+ }
+ boolean isChildSchemaExists = null != this.childSchema;
+ out.writeBoolean(isChildSchemaExists);
+ if (isChildSchemaExists) {
+ this.childSchema.write(out);
+ }
+ if (properties == null) {
+ out.writeShort(0);
+ } else {
+ out.writeShort(properties.size());
+ for (Map.Entry<String, String> entry : properties.entrySet()) {
+ out.writeUTF(entry.getKey());
+ out.writeUTF(entry.getValue());
+ }
+ }
+ }
+
+ @Override public void readFields(DataInput in) throws IOException {
+ this.className = in.readUTF();
+ boolean isRelationIdnentifierExists = in.readBoolean();
+ if (isRelationIdnentifierExists) {
+ this.relationIdentifier = new RelationIdentifier(null, null, null);
+ this.relationIdentifier.readFields(in);
+ }
+ boolean isChildSchemaExists = in.readBoolean();
+ if (isChildSchemaExists) {
+ this.childSchema = new TableSchema();
+ this.childSchema.readFields(in);
+ }
+
+ int mapSize = in.readShort();
+ this.properties = new HashMap<>(mapSize);
+ for (int i = 0; i < mapSize; i++) {
+ String key = in.readUTF();
+ String value = in.readUTF();
+ this.properties.put(key, value);
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
new file mode 100644
index 0000000..9a70b8b
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.metadata.schema.table;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * class to maintain the relation between parent and child
+ */
+public class RelationIdentifier implements Serializable, Writable {
+
+ private String databaseName;
+
+ private String tableName;
+
+ private String tableId;
+
+ public RelationIdentifier(String databaseName, String tableName, String tableId) {
+ this.databaseName = databaseName;
+ this.tableName = tableName;
+ this.tableId = tableId;
+ }
+
+ public String getDatabaseName() {
+ return databaseName;
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
+ public String getTableId() {
+ return tableId;
+ }
+
+ @Override public void write(DataOutput out) throws IOException {
+ out.writeUTF(databaseName);
+ out.writeUTF(tableName);
+ out.writeUTF(tableId);
+ }
+
+ @Override public void readFields(DataInput in) throws IOException {
+ this.databaseName = in.readUTF();
+ this.tableName = in.readUTF();
+ this.tableId = in.readUTF();
+ }
+
+ @Override public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ RelationIdentifier that = (RelationIdentifier) o;
+
+ if (databaseName != null ?
+ !databaseName.equals(that.databaseName) :
+ that.databaseName != null) {
+ return false;
+ }
+ if (tableName != null ? !tableName.equals(that.tableName) : that.tableName != null) {
+ return false;
+ }
+ return tableId != null ? tableId.equals(that.tableId) : that.tableId == null;
+ }
+
+ @Override public int hashCode() {
+ int result = databaseName != null ? databaseName.hashCode() : 0;
+ result = 31 * result + (tableName != null ? tableName.hashCode() : 0);
+ result = 31 * result + (tableId != null ? tableId.hashCode() : 0);
+ return result;
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index d1a7e5b..3acd6d6 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -24,13 +24,15 @@ import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.Serializable;
-import java.util.Map;
+import java.util.*;
import org.apache.carbondata.common.logging.LogService;
import org.apache.carbondata.common.logging.LogServiceFactory;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
/**
* Store the information about the table.
@@ -79,6 +81,14 @@ public class TableInfo implements Serializable, Writable {
// this idenifier is a lazy field which will be created when it is used first time
private AbsoluteTableIdentifier identifier;
+ private List<DataMapSchema> dataMapSchemaList;
+
+ private List<RelationIdentifier> parentRelationIdentifiers;
+
+ public TableInfo() {
+ dataMapSchemaList = new ArrayList<>();
+ }
+
/**
* @return the factTable
*/
@@ -91,6 +101,23 @@ public class TableInfo implements Serializable, Writable {
*/
public void setFactTable(TableSchema factTable) {
this.factTable = factTable;
+ updateParentRelationIdentifier();
+ }
+
+ private void updateParentRelationIdentifier() {
+ Set<RelationIdentifier> parentRelationIdentifiers = new HashSet<>();
+ this.parentRelationIdentifiers = new ArrayList<>();
+ List<ColumnSchema> listOfColumns = this.factTable.getListOfColumns();
+ for (ColumnSchema columnSchema : listOfColumns) {
+ List<ParentColumnTableRelation> parentColumnTableRelations =
+ columnSchema.getParentColumnTableRelations();
+ if (null != parentColumnTableRelations) {
+ for (int i = 0; i < parentColumnTableRelations.size(); i++) {
+ parentRelationIdentifiers.add(parentColumnTableRelations.get(i).getRelationIdentifier());
+ }
+ }
+ }
+ this.parentRelationIdentifiers.addAll(parentRelationIdentifiers);
}
/**
@@ -157,6 +184,14 @@ public class TableInfo implements Serializable, Writable {
this.storePath = storePath;
}
+ public List<DataMapSchema> getDataMapSchemaList() {
+ return dataMapSchemaList;
+ }
+
+ public void setDataMapSchemaList(List<DataMapSchema> dataMapSchemaList) {
+ this.dataMapSchemaList = dataMapSchemaList;
+ }
+
/**
* to generate the hash code
*/
@@ -225,10 +260,27 @@ public class TableInfo implements Serializable, Writable {
out.writeLong(lastUpdatedTime);
out.writeUTF(metaDataFilepath);
out.writeUTF(storePath);
+ boolean isChildSchemaExists =
+ null != dataMapSchemaList && dataMapSchemaList.size() > 0 ? true : false;
+ out.writeBoolean(isChildSchemaExists);
+ if (isChildSchemaExists) {
+ out.writeShort(dataMapSchemaList.size());
+ for (int i = 0; i < dataMapSchemaList.size(); i++) {
+ dataMapSchemaList.get(i).write(out);
+ }
+ }
+ boolean isParentTableRelationIndentifierExists =
+ null != parentRelationIdentifiers && parentRelationIdentifiers.size() > 0;
+ out.writeBoolean(isParentTableRelationIndentifierExists);
+ if (isParentTableRelationIndentifierExists) {
+ out.writeShort(parentRelationIdentifiers.size());
+ for (int i = 0; i < parentRelationIdentifiers.size(); i++) {
+ parentRelationIdentifiers.get(i).write(out);
+ }
+ }
}
- @Override
- public void readFields(DataInput in) throws IOException {
+ @Override public void readFields(DataInput in) throws IOException {
this.databaseName = in.readUTF();
this.tableUniqueName = in.readUTF();
this.factTable = new TableSchema();
@@ -236,6 +288,26 @@ public class TableInfo implements Serializable, Writable {
this.lastUpdatedTime = in.readLong();
this.metaDataFilepath = in.readUTF();
this.storePath = in.readUTF();
+ boolean isChildSchemaExists = in.readBoolean();
+ this.dataMapSchemaList = new ArrayList<>();
+ if (isChildSchemaExists) {
+ short numberOfChildTable = in.readShort();
+ for (int i = 0; i < numberOfChildTable; i++) {
+ DataMapSchema childSchema = new DataMapSchema(null);
+ childSchema.readFields(in);
+ dataMapSchemaList.add(childSchema);
+ }
+ }
+ boolean isParentTableRelationIndentifierExists = in.readBoolean();
+ if (isParentTableRelationIndentifierExists) {
+ short parentTableIndentifiersListSize = in.readShort();
+ this.parentRelationIdentifiers = new ArrayList<>();
+ for (int i = 0; i < parentTableIndentifiersListSize; i++) {
+ RelationIdentifier relationIdentifier = new RelationIdentifier(null, null, null);
+ relationIdentifier.readFields(in);
+ this.parentRelationIdentifiers.add(relationIdentifier);
+ }
+ }
}
public AbsoluteTableIdentifier getOrCreateAbsoluteTableIdentifier() {
@@ -259,4 +331,9 @@ public class TableInfo implements Serializable, Writable {
tableInfo.readFields(in);
return tableInfo;
}
+
+ public List<RelationIdentifier> getParentRelationIdentifiers() {
+ return parentRelationIdentifiers;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
index 7dc41a4..5b013d1 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
@@ -21,6 +21,7 @@ import java.io.DataOutput;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -250,4 +251,29 @@ public class TableSchema implements Serializable, Writable {
}
}
+ /**
+ * Below method will be used to build child schema object which will be stored in
+ * parent table
+ *
+ * @param className
+ * @param databaseName
+ * @param queryString
+ * @param queryType
+ *
+ * @return datamap schema
+ */
+ public DataMapSchema buildChildSchema(String className, String databaseName, String queryString,
+ String queryType) {
+ RelationIdentifier relationIdentifier =
+ new RelationIdentifier(databaseName, tableName, tableId);
+ Map<String, String> properties = new HashMap<>();
+ properties.put("CHILD_SELECT QUERY", queryString);
+ properties.put("QUERYTYPE", queryType);
+ DataMapSchema dataMapSchema = new DataMapSchema(className);
+ dataMapSchema.setChildSchema(this);
+ dataMapSchema.setProperties(properties);
+ dataMapSchema.setRelationIdentifier(relationIdentifier);
+ return dataMapSchema;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
index cad3dd6..ea7005f 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
@@ -122,6 +122,13 @@ public class ColumnSchema implements Serializable, Writable {
private boolean isSortColumn = false;
/**
+ * aggregate function used in pre aggregate table
+ */
+ private String aggFunction = "";
+
+ private List<ParentColumnTableRelation> parentColumnTableRelations;
+
+ /**
* @return the columnName
*/
public String getColumnName() {
@@ -274,6 +281,15 @@ public class ColumnSchema implements Serializable, Writable {
this.defaultValue = defaultValue;
}
+ public List<ParentColumnTableRelation> getParentColumnTableRelations() {
+ return parentColumnTableRelations;
+ }
+
+ public void setParentColumnTableRelations(
+ List<ParentColumnTableRelation> parentColumnTableRelations) {
+ this.parentColumnTableRelations = parentColumnTableRelations;
+ }
+
/**
* hash code method to check get the hashcode based.
* for generating the hash code only column name and column unique id will considered
@@ -282,7 +298,7 @@ public class ColumnSchema implements Serializable, Writable {
final int prime = 31;
int result = 1;
result = prime * result + ((columnName == null) ? 0 : columnName.hashCode()) +
- ((dataType == null) ? 0 : dataType.hashCode());
+ ((dataType == null) ? 0 : dataType.hashCode());
return result;
}
@@ -415,6 +431,14 @@ public class ColumnSchema implements Serializable, Writable {
isSortColumn = sortColumn;
}
+ public String getAggFunction() {
+ return aggFunction;
+ }
+
+ public void setAggFunction(String aggFunction) {
+ this.aggFunction = aggFunction;
+ }
+
@Override
public void write(DataOutput out) throws IOException {
out.writeShort(dataType.getId());
@@ -451,6 +475,16 @@ public class ColumnSchema implements Serializable, Writable {
}
out.writeBoolean(invisible);
out.writeBoolean(isSortColumn);
+ out.writeUTF(null != aggFunction ? aggFunction : "");
+ boolean isParentTableColumnRelationExists =
+ null != parentColumnTableRelations && parentColumnTableRelations.size() > 0;
+ out.writeBoolean(isParentTableColumnRelationExists);
+ if (isParentTableColumnRelationExists) {
+ out.writeShort(parentColumnTableRelations.size());
+ for (int i = 0; i < parentColumnTableRelations.size(); i++) {
+ parentColumnTableRelations.get(i).write(out);
+ }
+ }
}
@Override
@@ -486,5 +520,17 @@ public class ColumnSchema implements Serializable, Writable {
}
this.invisible = in.readBoolean();
this.isSortColumn = in.readBoolean();
+ this.aggFunction = in.readUTF();
+ boolean isParentTableColumnRelationExists = in.readBoolean();
+ if (isParentTableColumnRelationExists) {
+ short parentColumnTableRelationSize = in.readShort();
+ this.parentColumnTableRelations = new ArrayList<>(parentColumnTableRelationSize);
+ for (int i = 0; i < parentColumnTableRelationSize; i++) {
+ ParentColumnTableRelation parentColumnTableRelation =
+ new ParentColumnTableRelation(null, null, null);
+ parentColumnTableRelation.readFields(in);
+ parentColumnTableRelations.add(parentColumnTableRelation);
+ }
+ }
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index ea8bcb2..b9ec3f1 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -39,7 +39,9 @@ import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.metadata.datatype.DecimalType;
import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
import org.apache.carbondata.core.reader.CarbonIndexFileReader;
import org.apache.carbondata.core.util.path.CarbonTablePath;
import org.apache.carbondata.format.BlockIndex;
@@ -287,9 +289,35 @@ public abstract class AbstractDataFileFooterConverter {
wrapperColumnSchema.setSortColumn(true);
}
}
+ wrapperColumnSchema.setAggFunction(externalColumnSchema.getAggregate_function());
+ List<org.apache.carbondata.format.ParentColumnTableRelation> parentColumnTableRelation =
+ externalColumnSchema.getParentColumnTableRelations();
+ if (null != parentColumnTableRelation) {
+ wrapperColumnSchema.setParentColumnTableRelations(
+ fromThriftToWrapperParentTableColumnRelations(parentColumnTableRelation));
+ }
return wrapperColumnSchema;
}
+ private List<ParentColumnTableRelation> fromThriftToWrapperParentTableColumnRelations(
+ List<org.apache.carbondata.format.ParentColumnTableRelation> thirftParentColumnRelation) {
+ List<ParentColumnTableRelation> parentColumnTableRelationList = new ArrayList<>();
+ for (org.apache.carbondata.format.ParentColumnTableRelation carbonTableRelation :
+ thirftParentColumnRelation) {
+ RelationIdentifier relationIdentifier =
+ new RelationIdentifier(carbonTableRelation.getRelationIdentifier().getDatabaseName(),
+ carbonTableRelation.getRelationIdentifier().getTableName(),
+ carbonTableRelation.getRelationIdentifier().getTableId());
+ ParentColumnTableRelation parentColumnTableRelation =
+ new ParentColumnTableRelation(relationIdentifier, carbonTableRelation.getColumnId(),
+ carbonTableRelation.getColumnName());
+ parentColumnTableRelationList.add(parentColumnTableRelation);
+ }
+ return parentColumnTableRelationList;
+ }
+
+
+
/**
* Below method is convert the thrift encoding to wrapper encoding
*
http://git-wip-us.apache.org/repos/asf/carbondata/blob/31699185/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 216d91f..4e9bb21 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -116,6 +116,12 @@ struct ColumnSchema{
* It will have column order which user has provided
*/
16: optional i32 schemaOrdinal
+
+ /**
+ * to maintain the column relation with parent table.
+ * will be usefull in case of pre-aggregate
+ **/
+ 17: optional list<ParentColumnTableRelation> parentColumnTableRelations;
}
/**
@@ -168,7 +174,32 @@ struct TableSchema{
6: optional PartitionInfo partitionInfo; // Partition information
}
+struct RelationIdentifier {
+ 1: optional string databaseName;
+ 2: required string tableName;
+ 3: required string tableId;
+}
+
+struct ParentColumnTableRelation {
+ 1: required RelationIdentifier relationIdentifier;
+ 2: required string columnId;
+ 3: required string columnName
+}
+
+struct DataMapSchema {
+ // class name
+ 1: required string className;
+ // relation indentifier
+ 2: optional RelationIdentifier relationIdentifire;
+ // in case of preaggregate it will be used to maintain the child schema
+ // which will be usefull in case of query and data load
+ 3: optional TableSchema childTableSchema;
+ // to maintain properties like select query, query type like groupby, join
+ 4: optional map<string, string> properties;
+}
+
struct TableInfo{
1: required TableSchema fact_table;
2: required list<TableSchema> aggregate_table_list;
+ 3: optional list<DataMapSchema> dataMapSchemas; // childSchema information
}
[6/9] carbondata git commit:
[CARBONDATA-1524][CARBONDATA-1525][AggTable] Added support for aggregate
table drop
Posted by ra...@apache.org.
[CARBONDATA-1524][CARBONDATA-1525][AggTable] Added support for aggregate table drop
This closes #1443
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/f7f516ef
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/f7f516ef
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/f7f516ef
Branch: refs/heads/master
Commit: f7f516ef665c98e43fce0427c40da933bb6f3185
Parents: 3d1d1ce
Author: kunal642 <ku...@gmail.com>
Authored: Wed Oct 18 20:09:04 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530
----------------------------------------------------------------------
.../table/column/ParentColumnTableRelation.java | 2 +
.../preaggregate/TestPreAggregateDrop.scala | 67 ++++++++++++++++++++
.../carbondata/events/DropTableEvents.scala | 6 +-
.../org/apache/carbondata/events/Events.scala | 2 +-
.../command/carbonTableSchemaCommon.scala | 2 +-
.../command/CarbonDropTableCommand.scala | 29 ++++++---
.../CreatePreAggregateTableCommand.scala | 4 +-
.../DropPreAggregateTablePostListener.scala | 49 ++++++++++++++
.../preaaggregate/PreAggregateUtil.scala | 26 ++++----
.../spark/sql/hive/CarbonFileMetastore.scala | 53 ++++++++++++++--
.../spark/sql/hive/CarbonHiveMetaStore.scala | 13 +++-
.../spark/sql/hive/CarbonSessionState.scala | 13 ++++
12 files changed, 231 insertions(+), 35 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
index 425d0f2..28dc12c 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
@@ -29,6 +29,8 @@ import org.apache.carbondata.core.metadata.schema.table.Writable;
*/
public class ParentColumnTableRelation implements Serializable, Writable {
+ private static final long serialVersionUID = 1321746085997166646L;
+
private RelationIdentifier relationIdentifier;
/**
* parent column id
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
new file mode 100644
index 0000000..4dad3e1
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.integration.spark.testsuite.preaggregate
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestPreAggregateDrop extends QueryTest with BeforeAndAfterAll {
+
+ override def beforeAll {
+ sql("drop table if exists maintable")
+ sql("drop table if exists preagg1")
+ sql("drop table if exists preagg2")
+ sql("create table maintable (a string, b string, c string) stored by 'carbondata'")
+ }
+
+ test("create and drop preaggregate table") {
+ sql(
+ "create table preagg1 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+ " a,sum(b) from maintable group by a")
+ sql("drop table if exists preagg1")
+ checkExistence(sql("show tables"), false, "preagg1")
+ }
+
+ test("dropping 1 aggregate table should not drop others") {
+ sql(
+ "create table preagg1 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+ " a,sum(b) from maintable group by a")
+ sql(
+ "create table preagg2 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+ " a,sum(c) from maintable group by a")
+ sql("drop table if exists preagg2")
+ val showTables = sql("show tables")
+ checkExistence(showTables, false, "preagg2")
+ checkExistence(showTables, true, "preagg1")
+ }
+
+ test("drop main table and check if preaggreagte is deleted") {
+ sql(
+ "create table preagg2 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+ " a,sum(c) from maintable group by a")
+ sql("drop table if exists maintable")
+ checkExistence(sql("show tables"), false, "preagg1", "maintable", "preagg2")
+ }
+
+ override def afterAll() {
+ sql("drop table if exists maintable")
+ sql("drop table if exists preagg1")
+ sql("drop table if exists preagg2")
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
index ed43de6..ab77fba 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
@@ -27,7 +27,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
* @param ifExistsSet
* @param sparkSession
*/
-case class DropTablePreEvent(carbonTable: CarbonTable,
+case class DropTablePreEvent(carbonTable: Option[CarbonTable],
ifExistsSet: Boolean,
sparkSession: SparkSession)
extends Event with DropTableEventInfo
@@ -39,7 +39,7 @@ case class DropTablePreEvent(carbonTable: CarbonTable,
* @param ifExistsSet
* @param sparkSession
*/
-case class DropTablePostEvent(carbonTable: CarbonTable,
+case class DropTablePostEvent(carbonTable: Option[CarbonTable],
ifExistsSet: Boolean,
sparkSession: SparkSession)
extends Event with DropTableEventInfo
@@ -51,7 +51,7 @@ case class DropTablePostEvent(carbonTable: CarbonTable,
* @param ifExistsSet
* @param sparkSession
*/
-case class DropTableAbortEvent(carbonTable: CarbonTable,
+case class DropTableAbortEvent(carbonTable: Option[CarbonTable],
ifExistsSet: Boolean,
sparkSession: SparkSession)
extends Event with DropTableEventInfo
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
index 0d923ed..4f8d57e 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
@@ -57,7 +57,7 @@ trait LookupRelationEventInfo {
* event for drop table
*/
trait DropTableEventInfo {
- val carbonTable: CarbonTable
+ val carbonTable: Option[CarbonTable]
val ifExistsSet: Boolean
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 37ba8a5..759d3d8 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -428,7 +428,7 @@ class TableNewProcessor(cm: TableModel) {
columnSchema.setSortColumn(false)
if(isParentColumnRelation) {
val dataMapField = map.get.get(field).get
- columnSchema.setAggFunction(dataMapField.aggregateFunction);
+ columnSchema.setAggFunction(dataMapField.aggregateFunction)
val relation = dataMapField.columnTableRelation.get
val parentColumnTableRelationList = new util.ArrayList[ParentColumnTableRelation]
val relationIdentifier = new RelationIdentifier(
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/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
index 5905493..a8e6c37 100644
--- 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
@@ -21,12 +21,14 @@ import scala.collection.mutable.ListBuffer
import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
import org.apache.spark.sql.hive.CarbonRelation
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, CarbonMetadata, 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.events.{DropTablePostEvent, DropTablePreEvent, OperationContext, OperationListenerBus}
@@ -60,9 +62,20 @@ case class CarbonDropTableCommand(
lock => carbonLocks += CarbonLockUtil.getLockObject(carbonTableIdentifier, lock)
}
LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
-
- // fires the event before dropping main table
- val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)
+ val carbonTable: Option[CarbonTable] =
+ catalog.getTableFromMetadataCache(dbName, tableName) match {
+ case Some(tableMeta) => Some(tableMeta.carbonTable)
+ case None => try {
+ Some(catalog.lookupRelation(identifier)(sparkSession)
+ .asInstanceOf[CarbonRelation].metaData.carbonTable)
+ } catch {
+ case ex: NoSuchTableException =>
+ if (!ifExistsSet) {
+ throw ex
+ }
+ None
+ }
+ }
val operationContext = new OperationContext
val dropTablePreEvent: DropTablePreEvent =
DropTablePreEvent(
@@ -70,23 +83,23 @@ case class CarbonDropTableCommand(
ifExistsSet,
sparkSession)
OperationListenerBus.getInstance.fireEvent(dropTablePreEvent, operationContext)
-
CarbonEnv.getInstance(sparkSession).carbonMetastore
.dropTable(tableIdentifier.getTablePath, identifier)(sparkSession)
+ // fires the event after dropping main table
val dropTablePostEvent: DropTablePostEvent =
DropTablePostEvent(
carbonTable,
ifExistsSet,
sparkSession)
- OperationListenerBus.getInstance.fireEvent(dropTablePreEvent, operationContext)
-
+ OperationListenerBus.getInstance.fireEvent(dropTablePostEvent, operationContext)
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 {
+ sys.error(s"Dropping table $dbName.$tableName failed: ${ ex.getMessage }")
+ }
+ finally {
if (carbonLocks.nonEmpty) {
val unlocked = carbonLocks.forall(_.unlock())
if (unlocked) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index ca384f9..e42e933 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -24,7 +24,7 @@ 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.{RelationIdentifier, TableInfo}
+import org.apache.carbondata.core.metadata.schema.table.TableInfo
import org.apache.carbondata.core.util.CarbonUtil
/**
@@ -132,5 +132,3 @@ case class CreatePreAggregateTableCommand(
Seq.empty
}
}
-
-
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
new file mode 100644
index 0000000..7127c46
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
@@ -0,0 +1,49 @@
+/*
+ * 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.preaaggregate
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.execution.command.CarbonDropTableCommand
+
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+import org.apache.carbondata.events.{DropTablePostEvent, Event, OperationContext, OperationEventListener}
+
+class DropPreAggregateTablePostListener extends OperationEventListener {
+
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val dropPostEvent = event.asInstanceOf[DropTablePostEvent]
+ val carbonTable = dropPostEvent.carbonTable
+ val sparkSession = dropPostEvent.sparkSession
+ if (carbonTable.isDefined && carbonTable.get.getTableInfo.getDataMapSchemaList != null &&
+ !carbonTable.get.getTableInfo.getDataMapSchemaList.isEmpty) {
+ val childSchemas = carbonTable.get.getTableInfo.getDataMapSchemaList
+ for (childSchema: DataMapSchema <- childSchemas.asScala) {
+ CarbonDropTableCommand(ifExistsSet = true,
+ Some(childSchema.getRelationIdentifier.getDatabaseName),
+ childSchema.getRelationIdentifier.getTableName).run(sparkSession)
+ }
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index c4b6783..fd0e543 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -59,8 +59,8 @@ object PreAggregateUtil {
/**
* Below method will be used to validate the select plan
* and get the required fields from select plan
- * Currently only aggregate query is support any other type of query will
- * fail
+ * Currently only aggregate query is support any other type of query will fail
+ *
* @param plan
* @param selectStmt
* @return list of fields
@@ -89,11 +89,11 @@ object PreAggregateUtil {
throw new MalformedCarbonCommandException(
"Distinct is not supported On Pre Aggregation")
}
- fieldToDataMapFieldMap ++= ((validateAggregateFunctionAndGetFields(carbonTable,
+ fieldToDataMapFieldMap ++= (validateAggregateFunctionAndGetFields(carbonTable,
attr.aggregateFunction,
parentTableName,
parentDatabaseName,
- parentTableId)))
+ parentTableId))
case attr: AttributeReference =>
fieldToDataMapFieldMap += getField(attr.name,
attr.dataType,
@@ -124,6 +124,7 @@ object PreAggregateUtil {
* in case of any other aggregate function it will throw error
* In case of avg it will return two fields one for count
* and other of sum of that column to support rollup
+ *
* @param carbonTable
* @param aggFunctions
* @param parentTableName
@@ -220,6 +221,7 @@ object PreAggregateUtil {
/**
* Below method will be used to get the fields object for pre aggregate table
+ *
* @param columnName
* @param dataType
* @param aggregateType
@@ -256,8 +258,7 @@ object PreAggregateUtil {
precision = precision,
scale = scale,
rawSchema = rawSchema), dataMapField)
- }
- else {
+ } else {
(Field(column = actualColumnName,
dataType = Some(dataType.typeName),
name = Some(actualColumnName),
@@ -268,7 +269,8 @@ object PreAggregateUtil {
/**
* Below method will be used to update the main table about the pre aggregate table information
- * in case of any exption it will throw error so pre aggregate table creation will fail
+ * in case of any exception it will throw error so pre aggregate table creation will fail
+ *
* @param dbName
* @param tableName
* @param childSchema
@@ -304,9 +306,8 @@ object PreAggregateUtil {
val thriftTable = schemaConverter
.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
updateSchemaInfo(carbonTable,
- thriftTable)(sparkSession,
- sparkSession.sessionState.asInstanceOf[CarbonSessionState])
- LOGGER.info(s"Pre Aggeragte Parent table updated is successful for table $dbName.$tableName")
+ thriftTable)(sparkSession)
+ LOGGER.info(s"Parent table updated is successful for table $dbName.$tableName")
} catch {
case e: Exception =>
LOGGER.error(e, "Pre Aggregate Parent table update failed reverting changes")
@@ -321,14 +322,13 @@ object PreAggregateUtil {
/**
* Below method will be used to update the main table schema
+ *
* @param carbonTable
* @param thriftTable
* @param sparkSession
- * @param sessionState
*/
def updateSchemaInfo(carbonTable: CarbonTable,
- thriftTable: TableInfo)(sparkSession: SparkSession,
- sessionState: CarbonSessionState): Unit = {
+ thriftTable: TableInfo)(sparkSession: SparkSession): Unit = {
val dbName = carbonTable.getDatabaseName
val tableName = carbonTable.getFactTableName
CarbonEnv.getInstance(sparkSession).carbonMetastore
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index 51c7f3b..ac75fa7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -20,12 +20,14 @@ package org.apache.spark.sql.hive
import java.util.UUID
import java.util.concurrent.atomic.AtomicLong
+import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
-import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, SparkSession}
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.carbondata.common.logging.LogServiceFactory
@@ -34,11 +36,10 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.datamap.DataMapStoreManager
import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.fileoperations.FileWriteOperation
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{schema, AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema
import org.apache.carbondata.core.metadata.schema.table
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
import org.apache.carbondata.core.util.CarbonUtil
import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
import org.apache.carbondata.core.writer.ThriftWriter
@@ -449,6 +450,41 @@ class CarbonFileMetastore extends CarbonMetaStore {
}
}
+ protected def updateParentTableInfo(parentRelationIdentifier: RelationIdentifier,
+ childCarbonTable: CarbonTable)(sparkSession: SparkSession): Unit = {
+ val dbName = parentRelationIdentifier.getDatabaseName
+ val tableName = parentRelationIdentifier.getTableName
+ val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+ try {
+ val tableMeta = metaStore.getTableFromMetadataCache(dbName, tableName)
+ if (tableMeta.isDefined) {
+ val parentCarbonTable = tableMeta.get.carbonTable
+ val childSchemas = parentCarbonTable.getTableInfo.getDataMapSchemaList
+ if (childSchemas == null) {
+ throw UninitializedFieldError("Child schemas is not initialized")
+ }
+ val childSchemaIterator = childSchemas.iterator()
+ while (childSchemaIterator.hasNext) {
+ val childSchema = childSchemaIterator.next()
+ if (childSchema.getChildSchema.equals(childCarbonTable.getTableInfo.getFactTable)) {
+ childSchemaIterator.remove()
+ }
+ }
+ val schemaConverter = new ThriftWrapperSchemaConverterImpl
+ PreAggregateUtil
+ .updateSchemaInfo(parentCarbonTable,
+ schemaConverter
+ .fromWrapperToExternalTableInfo(parentCarbonTable.getTableInfo,
+ dbName,
+ tableName))(sparkSession)
+ }
+ } catch {
+ case ex: Exception =>
+ LOGGER.error(ex, s"Updating parent table $dbName.$tableName failed.")
+ throw ex
+ }
+ }
+
def dropTable(tablePath: String, tableIdentifier: TableIdentifier)
(sparkSession: SparkSession) {
val dbName = tableIdentifier.database.get
@@ -461,6 +497,14 @@ class CarbonFileMetastore extends CarbonMetaStore {
ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
}
val fileType = FileFactory.getFileType(metadataFilePath)
+ if (carbonTable != null) {
+ val parentRelations = carbonTable.getTableInfo.getParentRelationIdentifiers
+ if (parentRelations != null && !parentRelations.isEmpty) {
+ for (parentRelation: RelationIdentifier <- parentRelations.asScala) {
+ updateParentTableInfo(parentRelation, carbonTable)(sparkSession)
+ }
+ }
+ }
if (FileFactory.isFileExist(metadataFilePath, fileType)) {
// while drop we should refresh the schema modified time so that if any thing has changed
@@ -468,6 +512,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
checkSchemasModifiedTimeAndReloadTables(identifier.getStorePath)
removeTableFromMetadata(dbName, tableName)
+
updateSchemasUpdatedTime(touchSchemaFileSystemTime(identifier.getStorePath))
CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
// discard cached table info in cachedDataSourceTables
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
index c64b7bb..6bd80f3 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
@@ -18,14 +18,16 @@ package org.apache.spark.sql.hive
import scala.collection.JavaConverters._
-import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
import org.apache.carbondata.core.util.CarbonUtil
import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
import org.apache.carbondata.format
@@ -79,6 +81,12 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
ManageDictionaryAndBTree.clearBTreeAndDictionaryLRUCache(carbonTable)
}
checkSchemasModifiedTimeAndReloadTables(identifier.getStorePath)
+ val parentRelations = carbonTable.getTableInfo.getParentRelationIdentifiers
+ if (parentRelations != null && !parentRelations.isEmpty) {
+ for (parentRelation: RelationIdentifier <- parentRelations.asScala) {
+ updateParentTableInfo(parentRelation, carbonTable)(sparkSession)
+ }
+ }
removeTableFromMetadata(dbName, tableName)
CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
// discard cached table info in cachedDataSourceTables
@@ -107,6 +115,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
carbonTable.getFactTableName)
}
+
/**
* This method will overwrite the existing schema and update it with the given details
*
http://git-wip-us.apache.org/repos/asf/carbondata/blob/f7f516ef/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 9cad7b0..97ea7f8 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
@@ -26,6 +26,7 @@ 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.SparkOptimizer
+import org.apache.spark.sql.execution.command.preaaggregate.DropPreAggregateTablePostListener
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
import org.apache.spark.sql.internal.SQLConf
@@ -34,6 +35,7 @@ import org.apache.spark.sql.parser.CarbonSparkSqlParser
import org.apache.carbondata.core.datamap.DataMapStoreManager
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.events.{DropTablePostEvent, Event, OperationListenerBus}
/**
* This class will have carbon catalog and refresh the relation from cache if the carbontable in
@@ -124,6 +126,15 @@ class CarbonSessionCatalog(
}
}
+object CarbonSessionState {
+
+ def init(): Unit = {
+ OperationListenerBus.getInstance()
+ .addListener(classOf[DropTablePostEvent], new DropPreAggregateTablePostListener)
+ }
+
+}
+
/**
* Session state implementation to override sql parser and adding strategies
* @param sparkSession
@@ -140,6 +151,8 @@ class CarbonSessionState(sparkSession: SparkSession) extends HiveSessionState(sp
)
experimentalMethods.extraOptimizations = Seq(new CarbonLateDecodeRule)
+ CarbonSessionState.init()
+
override lazy val optimizer: Optimizer = new CarbonOptimizer(catalog, conf, experimentalMethods)
override lazy val analyzer: Analyzer = {
[9/9] carbondata git commit: [CARBONDATA-1520] [PreAgg] Support
pre-aggregate table load
Posted by ra...@apache.org.
[CARBONDATA-1520] [PreAgg] Support pre-aggregate table load
This closes #1446
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/cc0e6f1e
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/cc0e6f1e
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/cc0e6f1e
Branch: refs/heads/master
Commit: cc0e6f1e77b39d712de0e6101b2d24e57c5b47cb
Parents: f7f516e
Author: kunal642 <ku...@gmail.com>
Authored: Thu Oct 26 17:09:54 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530
----------------------------------------------------------------------
.../core/constants/CarbonCommonConstants.java | 6 +
.../core/indexstore/UnsafeMemoryDMStore.java | 14 +-
.../blockletindex/BlockletDataMap.java | 41 +++--
.../core/indexstore/row/DataMapRow.java | 6 +-
.../core/indexstore/row/DataMapRowImpl.java | 4 +-
.../core/indexstore/row/UnsafeDataMapRow.java | 8 +-
.../core/indexstore/schema/CarbonRowSchema.java | 124 +++++++++++++
.../core/indexstore/schema/DataMapSchema.java | 124 -------------
.../metadata/converter/SchemaConverter.java | 2 +-
.../core/metadata/schema/table/CarbonTable.java | 5 +
.../metadata/schema/table/DataMapSchema.java | 4 +-
.../schema/table/RelationIdentifier.java | 4 +
.../core/metadata/schema/table/TableInfo.java | 2 +-
.../core/scan/executor/util/QueryUtil.java | 44 +++--
.../carbondata/core/util/SessionParams.java | 13 ++
.../core/writer/CarbonIndexFileMergeWriter.java | 2 +-
.../carbondata/events/OperationListenerBus.java | 3 +-
.../hadoop/api/CarbonTableInputFormat.java | 19 +-
.../src/test/resources/sample.csv | 2 +
.../dataload/TestLoadDataGeneral.scala | 10 +-
.../TestLoadDataWithAutoLoadMerge.scala | 2 +-
.../TestLoadDataWithYarnLocalDirs.scala | 2 +-
.../preaggregate/TestPreAggregateLoad.scala | 172 +++++++++++++++++++
.../deleteTable/TestDeleteTableNewDDL.scala | 2 +
.../carbondata/spark/rdd/CarbonScanRDD.scala | 13 +-
.../spark/rdd/CarbonDataRDDFactory.scala | 8 +-
.../spark/sql/CarbonDictionaryDecoder.scala | 24 ++-
.../command/management/LoadTableCommand.scala | 4 +
.../CreatePreAggregateTableCommand.scala | 5 +
.../DropPreAggregateTablePostListener.scala | 49 ------
.../preaaggregate/PreAggregateListeners.scala | 81 +++++++++
.../preaaggregate/PreAggregateUtil.scala | 23 ++-
.../spark/sql/hive/CarbonAnalysisRules.scala | 57 +++++-
.../spark/sql/hive/CarbonSessionState.scala | 7 +-
.../execution/command/CarbonHiveCommands.scala | 3 +
.../impl/DictionaryFieldConverterImpl.java | 8 +-
.../converter/impl/FieldEncoderFactory.java | 38 +++-
.../loading/model/CarbonLoadModel.java | 13 ++
38 files changed, 685 insertions(+), 263 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 014478f..e27e5bd 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -57,6 +57,12 @@ public final class CarbonCommonConstants {
public static final String CARBON_INPUT_SEGMENTS = "carbon.input.segments.";
/**
+ * Fetch and validate the segments.
+ * Used for aggregate table load as segment validation is not required.
+ */
+ public static final String VALIDATE_CARBON_INPUT_SEGMENTS = "validate.carbon.input.segments.";
+
+ /**
* location of the carbon member, hierarchy and fact files
*/
@CarbonProperty
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
index bf1678a..450796a 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/UnsafeMemoryDMStore.java
@@ -18,7 +18,7 @@ package org.apache.carbondata.core.indexstore;
import org.apache.carbondata.core.indexstore.row.DataMapRow;
import org.apache.carbondata.core.indexstore.row.UnsafeDataMapRow;
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
import org.apache.carbondata.core.memory.MemoryBlock;
import org.apache.carbondata.core.memory.MemoryException;
import org.apache.carbondata.core.memory.UnsafeMemoryManager;
@@ -44,7 +44,7 @@ public class UnsafeMemoryDMStore {
private boolean isMemoryFreed;
- private DataMapSchema[] schema;
+ private CarbonRowSchema[] schema;
private int[] pointers;
@@ -52,7 +52,7 @@ public class UnsafeMemoryDMStore {
private final long taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
- public UnsafeMemoryDMStore(DataMapSchema[] schema) throws MemoryException {
+ public UnsafeMemoryDMStore(CarbonRowSchema[] schema) throws MemoryException {
this.schema = schema;
this.allocatedSize = capacity;
this.memoryBlock = UnsafeMemoryManager.allocateMemoryWithRetry(taskId, allocatedSize);
@@ -101,7 +101,7 @@ public class UnsafeMemoryDMStore {
pointers[rowCount++] = pointer;
}
- private void addToUnsafe(DataMapSchema schema, DataMapRow row, int index) {
+ private void addToUnsafe(CarbonRowSchema schema, DataMapRow row, int index) {
switch (schema.getSchemaType()) {
case FIXED:
DataType dataType = schema.getDataType();
@@ -154,8 +154,8 @@ public class UnsafeMemoryDMStore {
runningLength += data.length;
break;
case STRUCT:
- DataMapSchema[] childSchemas =
- ((DataMapSchema.StructDataMapSchema) schema).getChildSchemas();
+ CarbonRowSchema[] childSchemas =
+ ((CarbonRowSchema.StructCarbonRowSchema) schema).getChildSchemas();
DataMapRow struct = row.getRow(index);
for (int i = 0; i < childSchemas.length; i++) {
addToUnsafe(childSchemas[i], struct, i);
@@ -200,7 +200,7 @@ public class UnsafeMemoryDMStore {
return runningLength;
}
- public DataMapSchema[] getSchema() {
+ public CarbonRowSchema[] getSchema() {
return schema;
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index 7829034..43e265d 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
import org.apache.carbondata.core.indexstore.UnsafeMemoryDMStore;
import org.apache.carbondata.core.indexstore.row.DataMapRow;
import org.apache.carbondata.core.indexstore.row.DataMapRowImpl;
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
import org.apache.carbondata.core.keygenerator.KeyGenException;
import org.apache.carbondata.core.memory.MemoryException;
import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -132,7 +132,7 @@ public class BlockletDataMap implements DataMap, Cacheable {
String filePath) {
int[] minMaxLen = segmentProperties.getColumnsValueSize();
List<BlockletInfo> blockletList = fileFooter.getBlockletList();
- DataMapSchema[] schema = unsafeMemoryDMStore.getSchema();
+ CarbonRowSchema[] schema = unsafeMemoryDMStore.getSchema();
for (int index = 0; index < blockletList.size(); index++) {
DataMapRow row = new DataMapRowImpl(schema);
int ordinal = 0;
@@ -256,9 +256,10 @@ public class BlockletDataMap implements DataMap, Cacheable {
return updatedValues;
}
- private DataMapRow addMinMax(int[] minMaxLen, DataMapSchema dataMapSchema, byte[][] minValues) {
- DataMapSchema[] minSchemas =
- ((DataMapSchema.StructDataMapSchema) dataMapSchema).getChildSchemas();
+ private DataMapRow addMinMax(int[] minMaxLen, CarbonRowSchema carbonRowSchema,
+ byte[][] minValues) {
+ CarbonRowSchema[] minSchemas =
+ ((CarbonRowSchema.StructCarbonRowSchema) carbonRowSchema).getChildSchemas();
DataMapRow minRow = new DataMapRowImpl(minSchemas);
int minOrdinal = 0;
// min value adding
@@ -269,46 +270,48 @@ public class BlockletDataMap implements DataMap, Cacheable {
}
private void createSchema(SegmentProperties segmentProperties) throws MemoryException {
- List<DataMapSchema> indexSchemas = new ArrayList<>();
+ List<CarbonRowSchema> indexSchemas = new ArrayList<>();
// Index key
- indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
+ indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
int[] minMaxLen = segmentProperties.getColumnsValueSize();
// do it 2 times, one for min and one for max.
for (int k = 0; k < 2; k++) {
- DataMapSchema[] mapSchemas = new DataMapSchema[minMaxLen.length];
+ CarbonRowSchema[] mapSchemas = new CarbonRowSchema[minMaxLen.length];
for (int i = 0; i < minMaxLen.length; i++) {
if (minMaxLen[i] <= 0) {
- mapSchemas[i] = new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY);
+ mapSchemas[i] = new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY);
} else {
- mapSchemas[i] = new DataMapSchema.FixedDataMapSchema(DataTypes.BYTE_ARRAY, minMaxLen[i]);
+ mapSchemas[i] =
+ new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.BYTE_ARRAY, minMaxLen[i]);
}
}
- DataMapSchema mapSchema = new DataMapSchema.StructDataMapSchema(
- DataTypes.createDefaultStructType(), mapSchemas);
+ CarbonRowSchema mapSchema =
+ new CarbonRowSchema.StructCarbonRowSchema(DataTypes.createDefaultStructType(),
+ mapSchemas);
indexSchemas.add(mapSchema);
}
// for number of rows.
- indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.INT));
+ indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.INT));
// for table block path
- indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
+ indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
// for number of pages.
- indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.SHORT));
+ indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
// for version number.
- indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.SHORT));
+ indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.SHORT));
// for schema updated time.
- indexSchemas.add(new DataMapSchema.FixedDataMapSchema(DataTypes.LONG));
+ indexSchemas.add(new CarbonRowSchema.FixedCarbonRowSchema(DataTypes.LONG));
//for blocklet info
- indexSchemas.add(new DataMapSchema.VariableDataMapSchema(DataTypes.BYTE_ARRAY));
+ indexSchemas.add(new CarbonRowSchema.VariableCarbonRowSchema(DataTypes.BYTE_ARRAY));
unsafeMemoryDMStore =
- new UnsafeMemoryDMStore(indexSchemas.toArray(new DataMapSchema[indexSchemas.size()]));
+ new UnsafeMemoryDMStore(indexSchemas.toArray(new CarbonRowSchema[indexSchemas.size()]));
}
@Override
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
index 631e0ad..b764bdf 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRow.java
@@ -16,7 +16,7 @@
*/
package org.apache.carbondata.core.indexstore.row;
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
/**
* It is just a normal row to store data. Implementation classes could be safe and unsafe.
@@ -24,9 +24,9 @@ import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
*/
public abstract class DataMapRow {
- protected DataMapSchema[] schemas;
+ protected CarbonRowSchema[] schemas;
- public DataMapRow(DataMapSchema[] schemas) {
+ public DataMapRow(CarbonRowSchema[] schemas) {
this.schemas = schemas;
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
index 032b29e..0bb4a5c 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/DataMapRowImpl.java
@@ -16,7 +16,7 @@
*/
package org.apache.carbondata.core.indexstore.row;
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
/**
@@ -26,7 +26,7 @@ public class DataMapRowImpl extends DataMapRow {
private Object[] data;
- public DataMapRowImpl(DataMapSchema[] schemas) {
+ public DataMapRowImpl(CarbonRowSchema[] schemas) {
super(schemas);
this.data = new Object[schemas.length];
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java b/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
index 2c76990..932865d 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/row/UnsafeDataMapRow.java
@@ -17,7 +17,7 @@
package org.apache.carbondata.core.indexstore.row;
-import org.apache.carbondata.core.indexstore.schema.DataMapSchema;
+import org.apache.carbondata.core.indexstore.schema.CarbonRowSchema;
import org.apache.carbondata.core.memory.MemoryBlock;
import static org.apache.carbondata.core.memory.CarbonUnsafe.BYTE_ARRAY_OFFSET;
@@ -32,7 +32,7 @@ public class UnsafeDataMapRow extends DataMapRow {
private int pointer;
- public UnsafeDataMapRow(DataMapSchema[] schemas, MemoryBlock block, int pointer) {
+ public UnsafeDataMapRow(CarbonRowSchema[] schemas, MemoryBlock block, int pointer) {
super(schemas);
this.block = block;
this.pointer = pointer;
@@ -84,8 +84,8 @@ public class UnsafeDataMapRow extends DataMapRow {
}
@Override public DataMapRow getRow(int ordinal) {
- DataMapSchema[] childSchemas =
- ((DataMapSchema.StructDataMapSchema) schemas[ordinal]).getChildSchemas();
+ CarbonRowSchema[] childSchemas =
+ ((CarbonRowSchema.StructCarbonRowSchema) schemas[ordinal]).getChildSchemas();
return new UnsafeDataMapRow(childSchemas, block, pointer + getPosition(ordinal));
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/schema/CarbonRowSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/CarbonRowSchema.java b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/CarbonRowSchema.java
new file mode 100644
index 0000000..813be4a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/CarbonRowSchema.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.indexstore.schema;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+
+/**
+ * It just have 2 types right now, either fixed or variable.
+ */
+public abstract class CarbonRowSchema {
+
+ protected DataType dataType;
+
+ public CarbonRowSchema(DataType dataType) {
+ this.dataType = dataType;
+ }
+
+ /**
+ * Either fixed or variable length.
+ *
+ * @return
+ */
+ public DataType getDataType() {
+ return dataType;
+ }
+
+ /**
+ * Gives length in case of fixed schema other wise returns length
+ *
+ * @return
+ */
+ public abstract int getLength();
+
+ /**
+ * schema type
+ * @return
+ */
+ public abstract DataMapSchemaType getSchemaType();
+
+ /*
+ * It has always fixed length, length cannot be updated later.
+ * Usage examples : all primitive types like short, int etc
+ */
+ public static class FixedCarbonRowSchema extends CarbonRowSchema {
+
+ private int length;
+
+ public FixedCarbonRowSchema(DataType dataType) {
+ super(dataType);
+ }
+
+ public FixedCarbonRowSchema(DataType dataType, int length) {
+ super(dataType);
+ this.length = length;
+ }
+
+ @Override public int getLength() {
+ if (length == 0) {
+ return dataType.getSizeInBytes();
+ } else {
+ return length;
+ }
+ }
+
+ @Override public DataMapSchemaType getSchemaType() {
+ return DataMapSchemaType.FIXED;
+ }
+ }
+
+ public static class VariableCarbonRowSchema extends CarbonRowSchema {
+
+ public VariableCarbonRowSchema(DataType dataType) {
+ super(dataType);
+ }
+
+ @Override public int getLength() {
+ return dataType.getSizeInBytes();
+ }
+
+ @Override public DataMapSchemaType getSchemaType() {
+ return DataMapSchemaType.VARIABLE;
+ }
+ }
+
+ public static class StructCarbonRowSchema extends CarbonRowSchema {
+
+ private CarbonRowSchema[] childSchemas;
+
+ public StructCarbonRowSchema(DataType dataType, CarbonRowSchema[] childSchemas) {
+ super(dataType);
+ this.childSchemas = childSchemas;
+ }
+
+ @Override public int getLength() {
+ return dataType.getSizeInBytes();
+ }
+
+ public CarbonRowSchema[] getChildSchemas() {
+ return childSchemas;
+ }
+
+ @Override public DataMapSchemaType getSchemaType() {
+ return DataMapSchemaType.STRUCT;
+ }
+ }
+
+ public enum DataMapSchemaType {
+ FIXED, VARIABLE, STRUCT
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/indexstore/schema/DataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/indexstore/schema/DataMapSchema.java
deleted file mode 100644
index 80c68ac..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/schema/DataMapSchema.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.indexstore.schema;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-
-/**
- * It just have 2 types right now, either fixed or variable.
- */
-public abstract class DataMapSchema {
-
- protected DataType dataType;
-
- public DataMapSchema(DataType dataType) {
- this.dataType = dataType;
- }
-
- /**
- * Either fixed or variable length.
- *
- * @return
- */
- public DataType getDataType() {
- return dataType;
- }
-
- /**
- * Gives length in case of fixed schema other wise returns length
- *
- * @return
- */
- public abstract int getLength();
-
- /**
- * schema type
- * @return
- */
- public abstract DataMapSchemaType getSchemaType();
-
- /*
- * It has always fixed length, length cannot be updated later.
- * Usage examples : all primitive types like short, int etc
- */
- public static class FixedDataMapSchema extends DataMapSchema {
-
- private int length;
-
- public FixedDataMapSchema(DataType dataType) {
- super(dataType);
- }
-
- public FixedDataMapSchema(DataType dataType, int length) {
- super(dataType);
- this.length = length;
- }
-
- @Override public int getLength() {
- if (length == 0) {
- return dataType.getSizeInBytes();
- } else {
- return length;
- }
- }
-
- @Override public DataMapSchemaType getSchemaType() {
- return DataMapSchemaType.FIXED;
- }
- }
-
- public static class VariableDataMapSchema extends DataMapSchema {
-
- public VariableDataMapSchema(DataType dataType) {
- super(dataType);
- }
-
- @Override public int getLength() {
- return dataType.getSizeInBytes();
- }
-
- @Override public DataMapSchemaType getSchemaType() {
- return DataMapSchemaType.VARIABLE;
- }
- }
-
- public static class StructDataMapSchema extends DataMapSchema {
-
- private DataMapSchema[] childSchemas;
-
- public StructDataMapSchema(DataType dataType, DataMapSchema[] childSchemas) {
- super(dataType);
- this.childSchemas = childSchemas;
- }
-
- @Override public int getLength() {
- return dataType.getSizeInBytes();
- }
-
- public DataMapSchema[] getChildSchemas() {
- return childSchemas;
- }
-
- @Override public DataMapSchemaType getSchemaType() {
- return DataMapSchemaType.STRUCT;
- }
- }
-
- public enum DataMapSchemaType {
- FIXED, VARIABLE, STRUCT
- }
-}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
index bfbb6f7..af86253 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java
@@ -106,7 +106,7 @@ public interface SchemaConverter {
* method to convert thrift datamap schema object to wrapper
* data map object
* @param thriftchildSchema
- * @return DataMapSchema
+ * @return CarbonRowSchema
*/
DataMapSchema fromExternalToWrapperDataMapSchema(
org.apache.carbondata.format.DataMapSchema thriftchildSchema);
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index e63f4e3..4a6fb8b 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -702,4 +702,9 @@ public class CarbonTable implements Serializable {
this.dimensionOrdinalMax = dimensionOrdinalMax;
}
+ public boolean isPreAggregateTable() {
+ return tableInfo.getParentRelationIdentifiers() != null && !tableInfo
+ .getParentRelationIdentifiers().isEmpty();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
index 5f92ec8..9c71e37 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
@@ -28,6 +28,8 @@ import java.util.Map;
*/
public class DataMapSchema implements Serializable, Writable {
+ private static final long serialVersionUID = 6577149126264181553L;
+
private String className;
private RelationIdentifier relationIdentifier;
@@ -100,7 +102,7 @@ public class DataMapSchema implements Serializable, Writable {
this.className = in.readUTF();
boolean isRelationIdnentifierExists = in.readBoolean();
if (isRelationIdnentifierExists) {
- this.relationIdentifier = new RelationIdentifier(null, null, null);
+ this.relationIdentifier = new RelationIdentifier();
this.relationIdentifier.readFields(in);
}
boolean isChildSchemaExists = in.readBoolean();
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
index 9a70b8b..2a2d937 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
@@ -38,6 +38,10 @@ public class RelationIdentifier implements Serializable, Writable {
this.tableId = tableId;
}
+ public RelationIdentifier() {
+ this(null, null, null);
+ }
+
public String getDatabaseName() {
return databaseName;
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index 3acd6d6..44d8126 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -261,7 +261,7 @@ public class TableInfo implements Serializable, Writable {
out.writeUTF(metaDataFilepath);
out.writeUTF(storePath);
boolean isChildSchemaExists =
- null != dataMapSchemaList && dataMapSchemaList.size() > 0 ? true : false;
+ null != dataMapSchemaList && dataMapSchemaList.size() > 0;
out.writeBoolean(isChildSchemaExists);
if (isChildSchemaExists) {
out.writeShort(dataMapSchemaList.size());
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index b090e59..4d13462 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -42,6 +42,7 @@ import org.apache.carbondata.core.keygenerator.KeyGenerator;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.metadata.CarbonMetadata;
import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnIdentifier;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -353,14 +354,15 @@ public class QueryUtil {
CacheProvider cacheProvider = CacheProvider.getInstance();
Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider
.createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath());
-
List<Dictionary> columnDictionaryList =
forwardDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers);
Map<String, Dictionary> columnDictionaryMap = new HashMap<>(columnDictionaryList.size());
for (int i = 0; i < dictionaryColumnUniqueIdentifiers.size(); i++) {
// TODO: null check for column dictionary, if cache size is less it
// might return null here, in that case throw exception
- columnDictionaryMap.put(dictionaryColumnIdList.get(i), columnDictionaryList.get(i));
+ columnDictionaryMap
+ .put(dictionaryColumnUniqueIdentifiers.get(i).getColumnIdentifier().getColumnId(),
+ columnDictionaryList.get(i));
}
return columnDictionaryMap;
}
@@ -376,27 +378,47 @@ public class QueryUtil {
List<String> dictionaryColumnIdList, CarbonTableIdentifier carbonTableIdentifier,
TableProvider tableProvider) throws IOException {
CarbonTable carbonTable = tableProvider.getCarbonTable(carbonTableIdentifier);
- CarbonTablePath carbonTablePath =
- CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath(), carbonTableIdentifier);
List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
new ArrayList<>(dictionaryColumnIdList.size());
for (String columnId : dictionaryColumnIdList) {
CarbonDimension dimension = CarbonMetadata.getInstance()
.getCarbonDimensionBasedOnColIdentifier(carbonTable, columnId);
if (dimension != null) {
+ CarbonTableIdentifier newCarbonTableIdentifier;
+ ColumnIdentifier columnIdentifier;
+ if (null != dimension.getColumnSchema().getParentColumnTableRelations() && !dimension
+ .getColumnSchema().getParentColumnTableRelations().isEmpty()) {
+ newCarbonTableIdentifier = getTableIdentifierForColumn(dimension);
+ columnIdentifier = new ColumnIdentifier(
+ dimension.getColumnSchema().getParentColumnTableRelations().get(0).getColumnId(),
+ dimension.getColumnProperties(), dimension.getDataType());
+ } else {
+ newCarbonTableIdentifier = carbonTableIdentifier;
+ columnIdentifier = dimension.getColumnIdentifier();
+ }
+ CarbonTablePath newCarbonTablePath = CarbonStorePath
+ .getCarbonTablePath(carbonTable.getStorePath(), newCarbonTableIdentifier);
+
dictionaryColumnUniqueIdentifiers.add(
- new DictionaryColumnUniqueIdentifier(
- carbonTableIdentifier,
- dimension.getColumnIdentifier(),
- dimension.getDataType(),
- carbonTablePath
- )
- );
+ new DictionaryColumnUniqueIdentifier(newCarbonTableIdentifier, columnIdentifier,
+ dimension.getDataType(), newCarbonTablePath));
}
}
return dictionaryColumnUniqueIdentifiers;
}
+ public static CarbonTableIdentifier getTableIdentifierForColumn(CarbonDimension carbonDimension) {
+ String parentTableName =
+ carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
+ .getRelationIdentifier().getTableName();
+ String parentDatabaseName =
+ carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
+ .getRelationIdentifier().getDatabaseName();
+ String parentTableId = carbonDimension.getColumnSchema().getParentColumnTableRelations().get(0)
+ .getRelationIdentifier().getTableId();
+ return new CarbonTableIdentifier(parentDatabaseName, parentTableName, parentTableId);
+ }
+
/**
* Below method will used to get the method will be used to get the measure
* block indexes to be read from the file
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
index 1a91272..5dda9e4 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
@@ -66,6 +66,13 @@ public class SessionParams implements Serializable {
return sProps.get(key);
}
+ public String getProperty(String key, String defaultValue) {
+ if (!sProps.containsKey(key)) {
+ return defaultValue;
+ }
+ return sProps.get(key);
+ }
+
/**
* This method will be used to add a new property
*
@@ -172,6 +179,8 @@ public class SessionParams implements Serializable {
if (!isValid) {
throw new InvalidConfigurationException("Invalid CARBON_INPUT_SEGMENT_IDs");
}
+ } else if (key.startsWith(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS)) {
+ isValid = true;
} else {
throw new InvalidConfigurationException(
"The key " + key + " not supported for dynamic configuration.");
@@ -180,6 +189,10 @@ public class SessionParams implements Serializable {
return isValid;
}
+ public void removeProperty(String property) {
+ sProps.remove(property);
+ }
+
/**
* clear the set properties
*/
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
index 8d9bddc..e19ab24 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
@@ -49,7 +49,7 @@ public class CarbonIndexFileMergeWriter {
return file.getName().endsWith(CarbonTablePath.INDEX_FILE_EXT);
}
});
- if (indexFiles.length > 0) {
+ if (indexFiles != null && indexFiles.length > 0) {
SegmentIndexFileStore fileStore = new SegmentIndexFileStore();
fileStore.readAllIIndexOfSegment(segmentPath);
openThriftWriter(segmentPath + "/" +
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java b/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
index 01ecb04..321ddd5 100644
--- a/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
+++ b/core/src/main/java/org/apache/carbondata/events/OperationListenerBus.java
@@ -53,7 +53,7 @@ public class OperationListenerBus {
* @param eventClass
* @param operationEventListener
*/
- public void addListener(Class<? extends Event> eventClass,
+ public OperationListenerBus addListener(Class<? extends Event> eventClass,
OperationEventListener operationEventListener) {
String eventType = eventClass.getName();
@@ -63,6 +63,7 @@ public class OperationListenerBus {
eventMap.put(eventType, operationEventListeners);
}
operationEventListeners.add(operationEventListener);
+ return INSTANCE;
}
/**
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 57359fc..92ef6da 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -263,6 +263,14 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
configuration.set(CarbonTableInputFormat.VALIDATE_INPUT_SEGMENT_IDs, validate.toString());
}
+ public static void setAggeragateTableSegments(Configuration configuration, String segments) {
+ configuration.set(CarbonCommonConstants.CARBON_INPUT_SEGMENTS, segments);
+ }
+
+ private static String getAggeragateTableSegments(Configuration configuration) {
+ return configuration.get(CarbonCommonConstants.CARBON_INPUT_SEGMENTS);
+ }
+
/**
* get list of segment to access
*/
@@ -300,6 +308,11 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
@Override public List<InputSplit> getSplits(JobContext job) throws IOException {
AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
+ CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
+ if (null == carbonTable) {
+ throw new IOException("Missing/Corrupt schema file for table.");
+ }
+ String aggregateTableSegments = getAggeragateTableSegments(job.getConfiguration());
TableDataMap blockletMap =
DataMapStoreManager.getInstance().getDataMap(identifier, BlockletDataMap.NAME,
BlockletDataMapFactory.class.getName());
@@ -352,6 +365,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
if (invalidSegments.size() > 0) {
blockletMap.clear(invalidSegments);
}
+ } else {
+ filteredSegmentToAccess = Arrays.asList(aggregateTableSegments.split(","));
}
// Clean the updated segments from memory if the update happens on segments
@@ -376,12 +391,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
// process and resolve the expression
Expression filter = getFilterPredicates(job.getConfiguration());
- CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
TableProvider tableProvider = new SingleTableProvider(carbonTable);
// this will be null in case of corrupt schema file.
- if (null == carbonTable) {
- throw new IOException("Missing/Corrupt schema file for table.");
- }
PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName());
CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/resources/sample.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/sample.csv b/integration/spark-common-test/src/test/resources/sample.csv
index 7c57de7..06985e8 100644
--- a/integration/spark-common-test/src/test/resources/sample.csv
+++ b/integration/spark-common-test/src/test/resources/sample.csv
@@ -3,3 +3,5 @@ id,name,city,age
2,eason,shenzhen,27
3,jarry,wuhan,35
3,jarry,Bangalore,35
+4,kunal,Delhi,26
+4,vishal,Bangalore,29
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index b90a5ea..23d1292 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -65,7 +65,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
checkAnswer(
sql("SELECT COUNT(*) FROM loadtest"),
- Seq(Row(4))
+ Seq(Row(6))
)
}
@@ -74,7 +74,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
checkAnswer(
sql("SELECT COUNT(*) FROM loadtest"),
- Seq(Row(8))
+ Seq(Row(10))
)
}
@@ -83,7 +83,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
checkAnswer(
sql("SELECT COUNT(*) FROM loadtest"),
- Seq(Row(12))
+ Seq(Row(14))
)
}
@@ -92,7 +92,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest")
checkAnswer(
sql("SELECT COUNT(*) FROM loadtest"),
- Seq(Row(16))
+ Seq(Row(18))
)
}
@@ -101,7 +101,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
sql(s"LOAD DATA LOCAL INPATH '$testData' into table loadtest options ('delimiter'='\\017')")
checkAnswer(
sql("SELECT COUNT(*) FROM loadtest"),
- Seq(Row(20))
+ Seq(Row(22))
)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala
index 5211e9e..51e84d4 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithAutoLoadMerge.scala
@@ -44,7 +44,7 @@ class TestLoadDataWithAutoLoadMerge extends QueryTest with BeforeAndAfterAll {
sql(s"LOAD DATA LOCAL INPATH '$testData' into table automerge")
checkAnswer(
sql("SELECT COUNT(*) FROM automerge"),
- Seq(Row(4))
+ Seq(Row(6))
)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala
index e92a7fd..ff415ae 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithYarnLocalDirs.scala
@@ -87,7 +87,7 @@ class TestLoadDataWithYarnLocalDirs extends QueryTest with BeforeAndAfterAll {
disableMultipleDir
checkAnswer(sql("select id from carbontable_yarnLocalDirs"),
- Seq(Row(1), Row(2), Row(3), Row(3)))
+ Seq(Row(1), Row(2), Row(3), Row(3), Row(4), Row(4)))
cleanUpYarnLocalDir
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
new file mode 100644
index 0000000..0c65577
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.integration.spark.testsuite.preaggregate
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestPreAggregateLoad extends QueryTest with BeforeAndAfterAll {
+
+ val testData = s"$resourcesPath/sample.csv"
+
+ override def beforeAll(): Unit = {
+ sql("DROP TABLE IF EXISTS maintable")
+ }
+
+ private def createAllAggregateTables(parentTableName: String): Unit = {
+ sql(
+ s"""create table ${ parentTableName }_preagg_sum stored BY 'carbondata' tblproperties
+ |('parent'='$parentTableName') as select id,sum(age) from $parentTableName group by id"""
+ .stripMargin)
+ sql(
+ s"""create table ${ parentTableName }_preagg_avg stored BY 'carbondata' tblproperties
+ |('parent'='$parentTableName') as select id,avg(age) from $parentTableName group by id"""
+ .stripMargin)
+ sql(
+ s"""create table ${ parentTableName }_preagg_count stored BY 'carbondata' tblproperties
+ |('parent'='$parentTableName') as select id,count(age) from $parentTableName group by id"""
+ .stripMargin)
+ sql(
+ s"""create table ${ parentTableName }_preagg_min stored BY 'carbondata' tblproperties
+ |('parent'='$parentTableName') as select id,min(age) from $parentTableName group by id"""
+ .stripMargin)
+ sql(
+ s"""create table ${ parentTableName }_preagg_max stored BY 'carbondata' tblproperties
+ |('parent'='$parentTableName') as select id,max(age) from $parentTableName group by id"""
+ .stripMargin)
+ }
+
+ test("test load into main table with pre-aggregate table") {
+ sql("DROP TABLE IF EXISTS maintable")
+ sql(
+ """
+ | CREATE TABLE maintable(id int, name string, city string, age int)
+ | STORED BY 'org.apache.carbondata.format'
+ """.stripMargin)
+ createAllAggregateTables("maintable")
+ sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+ checkAnswer(sql(s"select * from maintable_preagg_sum"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+ checkAnswer(sql(s"select * from maintable_preagg_avg"),
+ Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55, 2)))
+ checkAnswer(sql(s"select * from maintable_preagg_count"),
+ Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+ checkAnswer(sql(s"select * from maintable_preagg_min"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
+ checkAnswer(sql(s"select * from maintable_preagg_max"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
+ sql("drop table if exists maintable")
+ }
+
+ test("test load into main table with pre-aggregate table with dictionary_include") {
+ sql("drop table if exists maintable")
+ sql(
+ """
+ | CREATE TABLE maintable(id int, name string, city string, age int)
+ | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
+ """.stripMargin)
+ createAllAggregateTables("maintable")
+ sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+ checkAnswer(sql(s"select * from maintable_preagg_sum"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+ checkAnswer(sql(s"select * from maintable_preagg_avg"),
+ Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55,2)))
+ checkAnswer(sql(s"select * from maintable_preagg_count"),
+ Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+ checkAnswer(sql(s"select * from maintable_preagg_min"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
+ checkAnswer(sql(s"select * from maintable_preagg_max"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
+ sql("drop table if exists maintable")
+ }
+
+ test("test load into main table with pre-aggregate table with single_pass") {
+ sql("drop table if exists maintable")
+ sql(
+ """
+ | CREATE TABLE maintable(id int, name string, city string, age int)
+ | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
+ """.stripMargin)
+ createAllAggregateTables("maintable")
+ sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable options('single_pass'='true')")
+ checkAnswer(sql(s"select * from maintable_preagg_sum"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+ checkAnswer(sql(s"select * from maintable_preagg_avg"),
+ Seq(Row(1, 31, 1), Row(2, 27, 1), Row(3, 70, 2), Row(4, 55,2)))
+ checkAnswer(sql(s"select * from maintable_preagg_count"),
+ Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+ checkAnswer(sql(s"select * from maintable_preagg_min"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 26)))
+ checkAnswer(sql(s"select * from maintable_preagg_max"),
+ Seq(Row(1, 31), Row(2, 27), Row(3, 35), Row(4, 29)))
+ sql("drop table if exists maintable")
+ }
+
+ test("test load into main table with incremental load") {
+ sql("drop table if exists maintable")
+ sql(
+ """
+ | CREATE TABLE maintable(id int, name string, city string, age int)
+ | STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('dictionary_include'='id')
+ """.stripMargin)
+ createAllAggregateTables("maintable")
+ sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+ sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+ checkAnswer(sql(s"select * from maintable_preagg_sum"),
+ Seq(Row(1, 31),
+ Row(2, 27),
+ Row(3, 70),
+ Row(4, 55),
+ Row(1, 31),
+ Row(2, 27),
+ Row(3, 70),
+ Row(4, 55)))
+ checkAnswer(sql(s"select * from maintable_preagg_avg"),
+ Seq(Row(1, 31, 1),
+ Row(2, 27, 1),
+ Row(3, 70, 2),
+ Row(4, 55, 2),
+ Row(1, 31, 1),
+ Row(2, 27, 1),
+ Row(3, 70, 2),
+ Row(4, 55, 2)))
+ checkAnswer(sql(s"select * from maintable_preagg_count"),
+ Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2), Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
+ checkAnswer(sql(s"select * from maintable_preagg_min"),
+ Seq(Row(1, 31),
+ Row(2, 27),
+ Row(3, 35),
+ Row(4, 26),
+ Row(1, 31),
+ Row(2, 27),
+ Row(3, 35),
+ Row(4, 26)))
+ checkAnswer(sql(s"select * from maintable_preagg_max"),
+ Seq(Row(1, 31),
+ Row(2, 27),
+ Row(3, 35),
+ Row(4, 29),
+ Row(1, 31),
+ Row(2, 27),
+ Row(3, 35),
+ Row(4, 29)))
+ sql("drop table if exists maintable")
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
index 485b94b..b82a0af 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
@@ -182,6 +182,7 @@ class TestDeleteTableNewDDL extends QueryTest with BeforeAndAfterAll {
test("drop table and create table with dictionary exclude string scenario") {
try {
+
sql("create database test")
sql(
"CREATE table test.dropTableTest3 (ID int, date String, country String, name " +
@@ -242,6 +243,7 @@ class TestDeleteTableNewDDL extends QueryTest with BeforeAndAfterAll {
sql("drop table if exists dropTableTest4")
sql("drop table if exists table1")
sql("drop table if exists table2")
+ sql("drop database if exists test cascade")
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 5e46417..52a31a9 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -41,7 +41,7 @@ import org.apache.carbondata.core.scan.expression.Expression
import org.apache.carbondata.core.scan.model.QueryModel
import org.apache.carbondata.core.stats.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
import org.apache.carbondata.core.statusmanager.FileFormat
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, DataTypeUtil, TaskMetricsMap}
+import org.apache.carbondata.core.util._
import org.apache.carbondata.hadoop._
import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
@@ -357,6 +357,17 @@ class CarbonScanRDD(
CarbonTableInputFormat
.setSegmentsToAccess(conf, segmentNumbersFromProperty.split(",").toList.asJava)
}
+ val carbonSessionInfo = ThreadLocalSessionInfo.getCarbonSessionInfo
+ if (carbonSessionInfo != null) {
+ CarbonTableInputFormat.setAggeragateTableSegments(conf, carbonSessionInfo.getSessionParams
+ .getProperty(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+ identifier.getCarbonTableIdentifier.getDatabaseName + "." +
+ identifier.getCarbonTableIdentifier.getTableName, ""))
+ CarbonTableInputFormat.setValidateSegmentsToAccess(conf, carbonSessionInfo.getSessionParams
+ .getProperty(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
+ identifier.getCarbonTableIdentifier.getDatabaseName + "." +
+ identifier.getCarbonTableIdentifier.getTableName, "true").toBoolean)
+ }
format
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 1083669..9899be1 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -265,8 +265,7 @@ object CarbonDataRDDFactory {
result: Option[DictionaryServer],
overwriteTable: Boolean,
dataFrame: Option[DataFrame] = None,
- updateModel: Option[UpdateTableModel] = None
- ): Unit = {
+ updateModel: Option[UpdateTableModel] = None): Unit = {
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
val operationContext = new OperationContext
// for handling of the segment Merging.
@@ -350,7 +349,7 @@ object CarbonDataRDDFactory {
if (value == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
} else if (value == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
- !loadStatus.equals(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
+ !loadStatus.equals(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)) {
loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
}
}
@@ -464,7 +463,8 @@ object CarbonDataRDDFactory {
throw new Exception(status(0)._2._2.errorMsg)
}
// if segment is empty then fail the data load
- if (!CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
+ if (!carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isPreAggregateTable &&
+ !CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
// update the load entry in table status file for changing the status to failure
CommonUtil.updateTableStatusForFailure(carbonLoadModel)
LOGGER.info("********starting clean up**********")
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 18f76d1..2671aad 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -34,10 +34,11 @@ import org.apache.spark.sql.types._
import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, ColumnIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
import org.apache.carbondata.core.metadata.datatype.{DataTypes => CarbonDataTypes}
import org.apache.carbondata.core.metadata.encoder.Encoding
import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.scan.executor.util.QueryUtil
import org.apache.carbondata.core.util.DataTypeUtil
import org.apache.carbondata.core.util.path.CarbonStorePath
import org.apache.carbondata.spark.CarbonAliasDecoderRelation
@@ -271,11 +272,24 @@ case class CarbonDictionaryDecoder(
case (tableName, columnIdentifier, carbonDimension) =>
if (columnIdentifier != null) {
try {
+ val (newCarbonTableIdentifier, newColumnIdentifier) =
+ if (null != carbonDimension.getColumnSchema.getParentColumnTableRelations &&
+ !carbonDimension
+ .getColumnSchema.getParentColumnTableRelations.isEmpty) {
+ (QueryUtil.getTableIdentifierForColumn(carbonDimension),
+ new ColumnIdentifier(carbonDimension.getColumnSchema
+ .getParentColumnTableRelations.get(0).getColumnId,
+ carbonDimension.getColumnProperties,
+ carbonDimension.getDataType))
+ } else {
+ (atiMap(tableName).getCarbonTableIdentifier, columnIdentifier)
+ }
val dictionaryColumnUniqueIdentifier = new DictionaryColumnUniqueIdentifier(
- atiMap(tableName).getCarbonTableIdentifier,
- columnIdentifier, carbonDimension.getDataType,
- CarbonStorePath.getCarbonTablePath(atiMap(tableName)))
- allDictIdentifiers += dictionaryColumnUniqueIdentifier;
+ newCarbonTableIdentifier,
+ newColumnIdentifier, carbonDimension.getDataType,
+ CarbonStorePath
+ .getCarbonTablePath(atiMap(tableName).getStorePath, newCarbonTableIdentifier))
+ allDictIdentifiers += dictionaryColumnUniqueIdentifier
new ForwardDictionaryWrapper(
storePath,
dictionaryColumnUniqueIdentifier)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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
index 222c30d..b28ec10 100644
--- 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
@@ -152,6 +152,10 @@ case class LoadTableCommand(
LOGGER.audit(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
carbonLoadModel.setUseOnePass(false)
}
+ // if table is an aggregate table then disable single pass.
+ if (carbonLoadModel.isAggLoadRequest) {
+ carbonLoadModel.setUseOnePass(false)
+ }
// Create table and metadata folders if not exist
val carbonTablePath = CarbonStorePath
.getCarbonTablePath(storePath, table.getCarbonTableIdentifier)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index e42e933..b952285 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -115,6 +115,11 @@ case class CreatePreAggregateTableCommand(
.buildChildSchema("", tableInfo.getDatabaseName, queryString, "AGGREGATION")
// upadting the parent table about child table
PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)
+ val loadAvailable = PreAggregateUtil
+ .checkMainTableLoad(parentTable)
+ if (loadAvailable) {
+ sparkSession.sql(s"insert into ${ cm.databaseName }.${ cm.tableName } $queryString")
+ }
} catch {
case e: Exception =>
val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
deleted file mode 100644
index 7127c46..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/DropPreAggregateTablePostListener.scala
+++ /dev/null
@@ -1,49 +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.preaaggregate
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.execution.command.CarbonDropTableCommand
-
-import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
-import org.apache.carbondata.events.{DropTablePostEvent, Event, OperationContext, OperationEventListener}
-
-class DropPreAggregateTablePostListener extends OperationEventListener {
-
- /**
- * Called on a specified event occurrence
- *
- * @param event
- */
- override def onEvent(event: Event, operationContext: OperationContext): Unit = {
- val dropPostEvent = event.asInstanceOf[DropTablePostEvent]
- val carbonTable = dropPostEvent.carbonTable
- val sparkSession = dropPostEvent.sparkSession
- if (carbonTable.isDefined && carbonTable.get.getTableInfo.getDataMapSchemaList != null &&
- !carbonTable.get.getTableInfo.getDataMapSchemaList.isEmpty) {
- val childSchemas = carbonTable.get.getTableInfo.getDataMapSchemaList
- for (childSchema: DataMapSchema <- childSchemas.asScala) {
- CarbonDropTableCommand(ifExistsSet = true,
- Some(childSchema.getRelationIdentifier.getDatabaseName),
- childSchema.getRelationIdentifier.getTableName).run(sparkSession)
- }
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
new file mode 100644
index 0000000..b507856
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -0,0 +1,81 @@
+/*
+ * 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.preaaggregate
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.CarbonSession
+import org.apache.spark.sql.execution.command.CarbonDropTableCommand
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+import org.apache.carbondata.events.{DropTablePostEvent, Event, LoadTablePostExecutionEvent, OperationContext, OperationEventListener}
+
+object DropPreAggregateTablePostListener extends OperationEventListener {
+
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val dropPostEvent = event.asInstanceOf[DropTablePostEvent]
+ val carbonTable = dropPostEvent.carbonTable
+ val sparkSession = dropPostEvent.sparkSession
+ if (carbonTable.isDefined && carbonTable.get.getTableInfo.getDataMapSchemaList != null &&
+ !carbonTable.get.getTableInfo.getDataMapSchemaList.isEmpty) {
+ val childSchemas = carbonTable.get.getTableInfo.getDataMapSchemaList
+ for (childSchema: DataMapSchema <- childSchemas.asScala) {
+ CarbonDropTableCommand(ifExistsSet = true,
+ Some(childSchema.getRelationIdentifier.getDatabaseName),
+ childSchema.getRelationIdentifier.getTableName).run(sparkSession)
+ }
+ }
+
+ }
+}
+
+object LoadPostAggregateListener extends OperationEventListener {
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val loadEvent = event.asInstanceOf[LoadTablePostExecutionEvent]
+ val sparkSession = loadEvent.sparkSession
+ val carbonLoadModel = loadEvent.carbonLoadModel
+ val table = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+ if (!table.getTableInfo.getDataMapSchemaList.isEmpty) {
+ for (dataMapSchema: DataMapSchema <- table.getTableInfo.getDataMapSchemaList.asScala) {
+ CarbonSession
+ .threadSet(CarbonCommonConstants.CARBON_INPUT_SEGMENTS +
+ carbonLoadModel.getDatabaseName + "." +
+ carbonLoadModel.getTableName,
+ carbonLoadModel.getSegmentId)
+ CarbonSession.threadSet(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS +
+ carbonLoadModel.getDatabaseName + "." +
+ carbonLoadModel.getTableName, "false")
+ val childTableName = dataMapSchema.getRelationIdentifier.getTableName
+ val childDatabaseName = dataMapSchema.getRelationIdentifier.getDatabaseName
+ val selectQuery = dataMapSchema.getProperties.get("CHILD_SELECT QUERY")
+ sparkSession.sql(s"insert into $childDatabaseName.$childTableName $selectQuery")
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index fd0e543..b35b525 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.command.{ColumnTableRelation, DataMapField, Field}
import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.sql.hive.CarbonRelation
import org.apache.spark.sql.hive.HiveExternalCatalog.{DATASOURCE_SCHEMA_NUMPARTS, DATASOURCE_SCHEMA_PART_PREFIX}
import org.apache.spark.sql.types.DataType
@@ -35,6 +35,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
import org.apache.carbondata.core.util.path.CarbonStorePath
import org.apache.carbondata.format.TableInfo
import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
@@ -428,4 +429,24 @@ object PreAggregateUtil {
thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
}
}
+
+ def getChildCarbonTable(databaseName: String, tableName: String)
+ (sparkSession: SparkSession): Option[CarbonTable] = {
+ val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+ metaStore.getTableFromMetadataCache(databaseName, tableName) match {
+ case Some(tableMeta) => Some(tableMeta.carbonTable)
+ case None => try {
+ Some(metaStore.lookupRelation(Some(databaseName), tableName)(sparkSession)
+ .asInstanceOf[CarbonRelation].metaData.carbonTable)
+ } catch {
+ case _: Exception =>
+ None
+ }
+ }
+ }
+
+ def checkMainTableLoad(carbonTable: CarbonTable): Boolean = {
+ SegmentStatusManager.readLoadMetadata(
+ carbonTable.getMetaDataFilepath).nonEmpty
+ }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 f61ab84..ba7e1eb 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,7 +20,8 @@ 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, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, ExprId, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{DeclarativeAggregate, _}
import org.apache.spark.sql.catalyst.plans.Inner
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
@@ -57,8 +58,16 @@ object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
.DEFAULT_MAX_NUMBER_OF_COLUMNS
)
}
- if (child.output.size >= relation.carbonRelation.output.size) {
- val newChildOutput = child.output.zipWithIndex.map { columnWithIndex =>
+ val isAggregateTable = !relation.carbonRelation.tableMeta.carbonTable.getTableInfo
+ .getParentRelationIdentifiers.isEmpty
+ // transform logical plan if the load is for aggregate table.
+ val childPlan = if (isAggregateTable) {
+ transformAggregatePlan(child)
+ } else {
+ child
+ }
+ if (childPlan.output.size >= relation.carbonRelation.output.size) {
+ val newChildOutput = childPlan.output.zipWithIndex.map { columnWithIndex =>
columnWithIndex._1 match {
case attr: Alias =>
Alias(attr.child, s"col${ columnWithIndex._2 }")(attr.exprId)
@@ -67,16 +76,54 @@ object CarbonPreInsertionCasts extends Rule[LogicalPlan] {
case attr => attr
}
}
- val newChild: LogicalPlan = if (newChildOutput == child.output) {
+ val newChild: LogicalPlan = if (newChildOutput == childPlan.output) {
p.child
} else {
- Project(newChildOutput, child)
+ Project(newChildOutput, childPlan)
}
InsertIntoCarbonTable(relation, p.partition, newChild, p.overwrite, p.ifNotExists)
} else {
sys.error("Cannot insert into target table because column number are different")
}
}
+
+ /**
+ * Transform the logical plan with average(col1) aggregation type to sum(col1) and count(col1).
+ *
+ * @param logicalPlan
+ * @return
+ */
+ private def transformAggregatePlan(logicalPlan: LogicalPlan): LogicalPlan = {
+ logicalPlan transform {
+ case aggregate@Aggregate(_, aExp, _) =>
+ val newExpressions = aExp flatMap {
+ case alias@Alias(attrExpression: AggregateExpression, _) =>
+ attrExpression.aggregateFunction flatMap {
+ case Average(attr: AttributeReference) =>
+ Seq(Alias(attrExpression
+ .copy(aggregateFunction = Sum(attr.withName(attr.name)),
+ resultId = NamedExpression.newExprId),
+ attr.name)(),
+ Alias(attrExpression
+ .copy(aggregateFunction = Count(attr.withName(attr.name)),
+ resultId = NamedExpression.newExprId), attr.name)())
+ case Average(Cast(attr: AttributeReference, _)) =>
+ Seq(Alias(attrExpression
+ .copy(aggregateFunction = Sum(attr.withName(attr.name)),
+ resultId = NamedExpression.newExprId),
+ attr.name)(),
+ Alias(attrExpression
+ .copy(aggregateFunction = Count(attr.withName(attr.name)),
+ resultId = NamedExpression.newExprId), attr.name)())
+ case _: DeclarativeAggregate => Seq(alias)
+ case _ => Nil
+ }
+ case namedExpr: NamedExpression => Seq(namedExpr)
+ }
+ aggregate.copy(aggregateExpressions = newExpressions)
+ case plan: LogicalPlan => plan
+ }
+ }
}
case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[LogicalPlan] {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 97ea7f8..d17dd11 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
@@ -26,7 +26,7 @@ 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.SparkOptimizer
-import org.apache.spark.sql.execution.command.preaaggregate.DropPreAggregateTablePostListener
+import org.apache.spark.sql.execution.command.preaaggregate.{DropPreAggregateTablePostListener, LoadPostAggregateListener}
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
import org.apache.spark.sql.internal.SQLConf
@@ -35,7 +35,7 @@ import org.apache.spark.sql.parser.CarbonSparkSqlParser
import org.apache.carbondata.core.datamap.DataMapStoreManager
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.events.{DropTablePostEvent, Event, OperationListenerBus}
+import org.apache.carbondata.events.{DropTablePostEvent, LoadTablePostExecutionEvent, OperationListenerBus}
/**
* This class will have carbon catalog and refresh the relation from cache if the carbontable in
@@ -130,7 +130,8 @@ object CarbonSessionState {
def init(): Unit = {
OperationListenerBus.getInstance()
- .addListener(classOf[DropTablePostEvent], new DropPreAggregateTablePostListener)
+ .addListener(classOf[DropTablePostEvent], DropPreAggregateTablePostListener)
+ .addListener(classOf[LoadTablePostExecutionEvent], LoadPostAggregateListener)
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/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 f2c8a0a..7d25efd 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
@@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.command._
import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, SessionParams}
+import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
@@ -83,6 +84,8 @@ object CarbonSetCommand {
"property should be in \" carbon.input.segments.<database_name>" +
".<table_name>=<seg_id list> \" format.")
}
+ } else if (key.startsWith(CarbonCommonConstants.VALIDATE_CARBON_INPUT_SEGMENTS)) {
+ sessionParams.addProperty(key.toLowerCase(), value)
}
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
index 2671393..7045101 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
@@ -37,7 +37,6 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
import org.apache.carbondata.core.util.CarbonUtil;
import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
import org.apache.carbondata.processing.loading.DataField;
import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
import org.apache.carbondata.processing.loading.dictionary.DictionaryServerClientDictionary;
@@ -68,15 +67,12 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
CarbonTableIdentifier carbonTableIdentifier, String nullFormat, int index,
DictionaryClient client, boolean useOnePass, String storePath,
- Map<Object, Integer> localCache, boolean isEmptyBadRecord) throws IOException {
+ Map<Object, Integer> localCache, boolean isEmptyBadRecord,
+ DictionaryColumnUniqueIdentifier identifier) throws IOException {
this.index = index;
this.carbonDimension = (CarbonDimension) dataField.getColumn();
this.nullFormat = nullFormat;
this.isEmptyBadRecord = isEmptyBadRecord;
- DictionaryColumnUniqueIdentifier identifier =
- new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
- dataField.getColumn().getColumnIdentifier(), dataField.getColumn().getDataType(),
- CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
// if use one pass, use DictionaryServerClientDictionary
if (useOnePass) {
[3/9] carbondata git commit: [CARBONDATA-1527] [CARBONDATA-1528]
[PreAgg] Restrict alter/update/delete for pre-aggregate table
Posted by ra...@apache.org.
[CARBONDATA-1527] [CARBONDATA-1528] [PreAgg] Restrict alter/update/delete for pre-aggregate table
This closes #1476
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/2b5faefa
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/2b5faefa
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/2b5faefa
Branch: refs/heads/master
Commit: 2b5faefada0d9078988f28b33249ebc3b2549c80
Parents: cc0e6f1
Author: kunal642 <ku...@gmail.com>
Authored: Mon Oct 23 18:28:15 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530
----------------------------------------------------------------------
.../core/metadata/schema/table/CarbonTable.java | 4 +
.../schema/table/RelationIdentifier.java | 12 ++
.../preaggregate/TestPreAggregateLoad.scala | 3 +-
.../iud/DeleteCarbonTableTestCase.scala | 15 ++
.../iud/UpdateCarbonTableTestCase.scala | 15 ++
.../testsuite/sortcolumns/TestSortColumns.scala | 3 +-
.../carbondata/events/AlterTableEvents.scala | 12 +-
.../org/apache/carbondata/events/Events.scala | 7 +-
.../management/DeleteLoadByIdCommand.scala | 2 +-
.../DeleteLoadByLoadDateCommand.scala | 6 +-
.../CreatePreAggregateTableCommand.scala | 2 +
.../preaaggregate/PreAggregateListeners.scala | 188 ++++++++++++++++++-
.../schema/CarbonAlterTableRenameCommand.scala | 2 +-
.../spark/sql/hive/CarbonSessionState.scala | 13 +-
.../AlterTableValidationTestCase.scala | 21 ++-
.../vectorreader/ChangeDataTypeTestCases.scala | 17 ++
.../vectorreader/DropColumnTestCases.scala | 16 ++
.../apache/spark/util/CarbonCommandSuite.scala | 18 +-
18 files changed, 339 insertions(+), 17 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 4a6fb8b..ca0952d 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -707,4 +707,8 @@ public class CarbonTable implements Serializable {
.getParentRelationIdentifiers().isEmpty();
}
+ public boolean hasPreAggregateTables() {
+ return tableInfo.getDataMapSchemaList() != null && !tableInfo
+ .getDataMapSchemaList().isEmpty();
+ }
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
index 2a2d937..c9c44bf 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
@@ -26,6 +26,18 @@ import java.io.Serializable;
*/
public class RelationIdentifier implements Serializable, Writable {
+ public void setDatabaseName(String databaseName) {
+ this.databaseName = databaseName;
+ }
+
+ public void setTableName(String tableName) {
+ this.tableName = tableName;
+ }
+
+ public void setTableId(String tableId) {
+ this.tableId = tableId;
+ }
+
private String databaseName;
private String tableName;
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
index 0c65577..1f576c5 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
@@ -19,8 +19,9 @@ package org.apache.carbondata.integration.spark.testsuite.preaggregate
import org.apache.spark.sql.Row
import org.apache.spark.sql.test.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
+import org.scalatest.{BeforeAndAfterAll, Ignore}
+@Ignore
class TestPreAggregateLoad extends QueryTest with BeforeAndAfterAll {
val testData = s"$resourcesPath/sample.csv"
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
index 3c2842c..a2bd6aa 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
@@ -129,6 +129,21 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
sql("DROP TABLE IF EXISTS default.carbon2")
}
+ test("test if delete is unsupported for pre-aggregate tables") {
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preagg1")
+ sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+ sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+ intercept[RuntimeException] {
+ sql("delete from preaggmain where a = 'abc'").show()
+ }.getMessage.contains("Delete operation is not supported for tables")
+ intercept[RuntimeException] {
+ sql("delete from preagg1 where preaggmain_a = 'abc'").show()
+ }.getMessage.contains("Delete operation is not supported for pre-aggregate table")
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preagg1")
+ }
+
override def afterAll {
sql("use default")
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index db289d9..4c43ec0 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -513,6 +513,21 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
sql("DROP TABLE IF EXISTS iud.rand")
}
+ test("test if update is unsupported for pre-aggregate tables") {
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preagg1")
+ sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+ sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+ intercept[RuntimeException] {
+ sql("update preaggmain set (a)=('a')").show
+ }.getMessage.contains("Update operation is not supported for tables")
+ intercept[RuntimeException] {
+ sql("update preagg1 set (a)=('a')").show
+ }.getMessage.contains("Update operation is not supported for pre-aggregate table")
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preagg1")
+ }
+
override def afterAll {
sql("use default")
sql("drop database if exists iud cascade")
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
index b5fd8a9..6c5aa55 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
@@ -35,7 +35,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
sql("CREATE TABLE tableOne(id int, name string, city string, age int) STORED BY 'org.apache.carbondata.format'")
sql("CREATE TABLE tableTwo(id int, age int) STORED BY 'org.apache.carbondata.format'")
sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/measureinsertintotest.csv' into table tableOne")
- sql("insert into table tableTwo select id, count(age) from tableOne group by id")
+
}
test("create table sort columns dictionary include - int") {
@@ -335,6 +335,7 @@ class TestSortColumns extends QueryTest with BeforeAndAfterAll {
}
test("Test tableTwo data") {
+ sql("insert into table tableTwo select id, count(age) from tableOne group by id")
checkAnswer(
sql("select id,age from tableTwo order by id"),
Seq(Row(1, 1), Row(2, 1), Row(3, 2), Row(4, 2)))
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
index ec79acc..2f7cf63 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/AlterTableEvents.scala
@@ -18,7 +18,7 @@ package org.apache.carbondata.events
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.SQLContext
-import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
+import org.apache.spark.sql.execution.command.{AlterTableDataTypeChangeModel, AlterTableDropColumnModel, AlterTableRenameModel}
import org.apache.carbondata.core.metadata.schema.table.CarbonTable
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -35,6 +35,16 @@ case class AlterTableDropColumnPreEvent(carbonTable: CarbonTable,
/**
+ * Class for handling clean up in case of any failure and abort the operation
+ *
+ * @param carbonTable
+ * @param alterTableDataTypeChangeModel
+ */
+case class AlterTableDataTypeChangePreEvent(carbonTable: CarbonTable,
+ alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel)
+ extends Event with AlterTableDataTypeChangeEventInfo
+
+/**
*
* @param carbonTable
* @param alterTableDropColumnModel
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
index 4f8d57e..9796dea 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/events/Events.scala
@@ -18,7 +18,7 @@
package org.apache.carbondata.events
import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, AlterTableRenameModel}
+import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableDataTypeChangeModel, AlterTableDropColumnModel, AlterTableRenameModel}
import org.apache.carbondata.core.metadata.CarbonTableIdentifier
import org.apache.carbondata.core.metadata.schema.table.CarbonTable
@@ -69,6 +69,11 @@ trait AlterTableDropColumnEventInfo {
val alterTableDropColumnModel: AlterTableDropColumnModel
}
+trait AlterTableDataTypeChangeEventInfo {
+ val carbonTable: CarbonTable
+ val alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel
+}
+
/**
* event for alter_table_rename
*/
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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
index 9ea4018..6a0465c 100644
--- 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
@@ -57,7 +57,7 @@ case class DeleteLoadByIdCommand(
DeleteSegmentByIdPostEvent(carbonTable,
loadIds,
sparkSession)
- OperationListenerBus.getInstance.fireEvent(deleteSegmentByIdPreEvent, operationContext)
+ OperationListenerBus.getInstance.fireEvent(deleteSegmentPostEvent, operationContext)
Seq.empty
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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
index 58d8236..83f41bb 100644
--- 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
@@ -51,14 +51,12 @@ case class DeleteLoadByLoadDateCommand(
loadDate,
GetDB.getDatabaseName(databaseNameOp, sparkSession),
tableName,
- carbonTable
- )
-
+ carbonTable)
val deleteSegmentPostEvent: DeleteSegmentByDatePostEvent =
DeleteSegmentByDatePostEvent(carbonTable,
loadDate,
sparkSession)
- OperationListenerBus.getInstance.fireEvent(deleteSegmentByDatePreEvent, operationContext)
+ OperationListenerBus.getInstance.fireEvent(deleteSegmentPostEvent, operationContext)
Seq.empty
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index b952285..e12cbb9 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -137,3 +137,5 @@ case class CreatePreAggregateTableCommand(
Seq.empty
}
}
+
+
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
index b507856..2ce97fe 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.command.CarbonDropTableCommand
import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
-import org.apache.carbondata.events.{DropTablePostEvent, Event, LoadTablePostExecutionEvent, OperationContext, OperationEventListener}
+import org.apache.carbondata.events._
object DropPreAggregateTablePostListener extends OperationEventListener {
@@ -79,3 +79,189 @@ object LoadPostAggregateListener extends OperationEventListener {
}
}
}
+
+object PreAggregateDataTypeChangePreListener extends OperationEventListener {
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ * @param operationContext
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val dataTypeChangePreListener = event.asInstanceOf[AlterTableDataTypeChangePreEvent]
+ val carbonTable = dataTypeChangePreListener.carbonTable
+ val alterTableDataTypeChangeModel = dataTypeChangePreListener.alterTableDataTypeChangeModel
+ val columnToBeAltered: String = alterTableDataTypeChangeModel.columnName
+ val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList
+ if (dataMapSchemas != null && !dataMapSchemas.isEmpty) {
+ dataMapSchemas.asScala.foreach { dataMapSchema =>
+ val childColumns = dataMapSchema.getChildSchema.getListOfColumns
+ if (childColumns.asScala.exists(_.getColumnName.equalsIgnoreCase(columnToBeAltered))) {
+ throw new UnsupportedOperationException(s"Column $columnToBeAltered exists in a " +
+ s"pre-aggregate table ${ dataMapSchema.toString
+ }. Cannot change datatype")
+ }
+ }
+
+ if (carbonTable.isPreAggregateTable) {
+ throw new UnsupportedOperationException(s"Cannot change data type for columns in " +
+ s"pre-aggreagate table ${
+ carbonTable.getDatabaseName
+ }.${ carbonTable.getFactTableName }")
+ }
+ }
+ }
+}
+
+object PreAggregateDeleteSegmentByDatePreListener extends OperationEventListener {
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ * @param operationContext
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val deleteSegmentByDatePreEvent = event.asInstanceOf[DeleteSegmentByDatePreEvent]
+ val carbonTable = deleteSegmentByDatePreEvent.carbonTable
+ if (carbonTable != null) {
+ if (carbonTable.hasPreAggregateTables) {
+ throw new UnsupportedOperationException(
+ "Delete segment operation is not supported on tables which have a pre-aggregate table. " +
+ "Drop pre-aggregation table to continue")
+ }
+ if (carbonTable.isPreAggregateTable) {
+ throw new UnsupportedOperationException(
+ "Delete segment operation is not supported on pre-aggregate table")
+ }
+ }
+ }
+}
+
+object PreAggregateDeleteSegmentByIdPreListener extends OperationEventListener {
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ * @param operationContext
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val tableEvent = event.asInstanceOf[DeleteSegmentByIdPreEvent]
+ val carbonTable = tableEvent.carbonTable
+ if (carbonTable != null) {
+ if (carbonTable.hasPreAggregateTables) {
+ throw new UnsupportedOperationException(
+ "Delete segment operation is not supported on tables which have a pre-aggregate table")
+ }
+ if (carbonTable.isPreAggregateTable) {
+ throw new UnsupportedOperationException(
+ "Delete segment operation is not supported on pre-aggregate table")
+ }
+ }
+ }
+
+}
+
+object PreAggregateDropColumnPreListener extends OperationEventListener {
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ * @param operationContext
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val dataTypeChangePreListener = event.asInstanceOf[AlterTableDropColumnPreEvent]
+ val carbonTable = dataTypeChangePreListener.carbonTable
+ val alterTableDropColumnModel = dataTypeChangePreListener.alterTableDropColumnModel
+ val columnsToBeDropped = alterTableDropColumnModel.columns
+ val dataMapSchemas = carbonTable.getTableInfo.getDataMapSchemaList
+ if (dataMapSchemas != null && !dataMapSchemas.isEmpty) {
+ dataMapSchemas.asScala.foreach { dataMapSchema =>
+ val parentColumnNames = dataMapSchema.getChildSchema.getListOfColumns.asScala
+ .flatMap(_.getParentColumnTableRelations.asScala.map(_.getColumnName))
+ val columnExistsInChild = parentColumnNames.collectFirst {
+ case parentColumnName if columnsToBeDropped.contains(parentColumnName) =>
+ parentColumnName
+ }
+ if (columnExistsInChild.isDefined) {
+ throw new UnsupportedOperationException(
+ s"Column ${ columnExistsInChild.head } cannot be dropped because it exists in a " +
+ s"pre-aggregate table ${ dataMapSchema.getRelationIdentifier.toString}")
+ }
+ }
+ if (carbonTable.isPreAggregateTable) {
+ throw new UnsupportedOperationException(s"Cannot drop columns in pre-aggreagate table ${
+ carbonTable.getDatabaseName}.${ carbonTable.getFactTableName }")
+ }
+ }
+ }
+}
+
+object PreAggregateRenameTablePreListener extends OperationEventListener {
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ * @param operationContext
+ */
+ override def onEvent(event: Event,
+ operationContext: OperationContext): Unit = {
+ val renameTablePostListener = event.asInstanceOf[AlterTableRenamePreEvent]
+ val carbonTable = renameTablePostListener.carbonTable
+ if (carbonTable.isPreAggregateTable) {
+ throw new UnsupportedOperationException(
+ "Rename operation for pre-aggregate table is not supported.")
+ }
+ if (carbonTable.hasPreAggregateTables) {
+ throw new UnsupportedOperationException(
+ "Rename operation is not supported for table with pre-aggregate tables")
+ }
+ }
+}
+
+object UpdatePreAggregatePreListener extends OperationEventListener {
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ * @param operationContext
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val tableEvent = event.asInstanceOf[UpdateTablePreEvent]
+ val carbonTable = tableEvent.carbonTable
+ if (carbonTable != null) {
+ if (carbonTable.hasPreAggregateTables) {
+ throw new UnsupportedOperationException(
+ "Update operation is not supported for tables which have a pre-aggregate table. Drop " +
+ "pre-aggregate tables to continue.")
+ }
+ if (carbonTable.isPreAggregateTable) {
+ throw new UnsupportedOperationException(
+ "Update operation is not supported for pre-aggregate table")
+ }
+ }
+ }
+}
+
+object DeletePreAggregatePreListener extends OperationEventListener {
+ /**
+ * Called on a specified event occurrence
+ *
+ * @param event
+ * @param operationContext
+ */
+ override def onEvent(event: Event, operationContext: OperationContext): Unit = {
+ val tableEvent = event.asInstanceOf[DeleteFromTablePreEvent]
+ val carbonTable = tableEvent.carbonTable
+ if (carbonTable != null) {
+ if (carbonTable.hasPreAggregateTables) {
+ throw new UnsupportedOperationException(
+ "Delete operation is not supported for tables which have a pre-aggregate table. Drop " +
+ "pre-aggregate tables to continue.")
+ }
+ if (carbonTable.isPreAggregateTable) {
+ throw new UnsupportedOperationException(
+ "Delete operation is not supported for pre-aggregate table")
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index e0617d6..b96baff 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -136,7 +136,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
alterTableRenameModel,
newTablePath,
sparkSession)
- OperationListenerBus.getInstance().fireEvent(alterTableRenamePreEvent, operationContext)
+ OperationListenerBus.getInstance().fireEvent(alterTableRenamePostEvent, operationContext)
sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
Some(oldDatabaseName)).quotedString)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/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 d17dd11..f698dd4 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
@@ -26,7 +26,7 @@ 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.SparkOptimizer
-import org.apache.spark.sql.execution.command.preaaggregate.{DropPreAggregateTablePostListener, LoadPostAggregateListener}
+import org.apache.spark.sql.execution.command.preaaggregate._
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy, StreamingTableStrategy}
import org.apache.spark.sql.internal.SQLConf
@@ -35,7 +35,7 @@ import org.apache.spark.sql.parser.CarbonSparkSqlParser
import org.apache.carbondata.core.datamap.DataMapStoreManager
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.events.{DropTablePostEvent, LoadTablePostExecutionEvent, OperationListenerBus}
+import org.apache.carbondata.events._
/**
* This class will have carbon catalog and refresh the relation from cache if the carbontable in
@@ -132,8 +132,15 @@ object CarbonSessionState {
OperationListenerBus.getInstance()
.addListener(classOf[DropTablePostEvent], DropPreAggregateTablePostListener)
.addListener(classOf[LoadTablePostExecutionEvent], LoadPostAggregateListener)
+ .addListener(classOf[DeleteSegmentByIdPreEvent], PreAggregateDeleteSegmentByIdPreListener)
+ .addListener(classOf[DeleteSegmentByDatePreEvent], PreAggregateDeleteSegmentByDatePreListener)
+ .addListener(classOf[UpdateTablePreEvent], UpdatePreAggregatePreListener)
+ .addListener(classOf[DeleteFromTablePreEvent], DeletePreAggregatePreListener)
+ .addListener(classOf[DeleteFromTablePreEvent], DeletePreAggregatePreListener)
+ .addListener(classOf[AlterTableDropColumnPreEvent], PreAggregateDropColumnPreListener)
+ .addListener(classOf[AlterTableRenamePreEvent], PreAggregateRenameTablePreListener)
+ .addListener(classOf[AlterTableDataTypeChangePreEvent], PreAggregateDataTypeChangePreListener)
}
-
}
/**
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
index 6f618fe..0274605 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
@@ -21,8 +21,11 @@ import java.io.File
import java.math.{BigDecimal, RoundingMode}
import java.sql.Timestamp
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.{CarbonEnv, Row}
import org.apache.spark.sql.common.util.Spark2QueryTest
+import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
+import org.apache.spark.sql.test.Spark2TestQueryExecutor
+import org.junit.Assert
import org.scalatest.BeforeAndAfterAll
import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -487,6 +490,22 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"empno,empname,role,doj")
}
+ test("test to check if new parent table name is reflected in pre-aggregate tables") {
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preaggmain_new")
+ sql("drop table if exists preagg1")
+ sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+ sql(
+ "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+ " a,sum(b) from PreAggMain group by a")
+ intercept[RuntimeException] {
+ sql("alter table preagg1 rename to preagg2")
+ }.getMessage.contains("Rename operation for pre-aggregate table is not supported.")
+ intercept[RuntimeException] {
+ sql("alter table preaggmain rename to preaggmain_new")
+ }.getMessage.contains("Rename operation is not supported for table with pre-aggregate tables")
+ }
+
override def afterAll {
sql("DROP TABLE IF EXISTS restructure")
sql("drop table if exists table1")
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
index 9f104ed..67ea21e 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
@@ -147,6 +147,23 @@ class ChangeDataTypeTestCases extends Spark2QueryTest with BeforeAndAfterAll {
test_change_int_to_long()
}
+ test("test data type change for with pre-aggregate table should throw exception") {
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preagg1")
+ sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+ sql(
+ "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+ " a,sum(b) from PreAggMain group by a")
+ intercept[RuntimeException] {
+ sql("alter table preaggmain drop columns(a)").show
+ }.getMessage.contains("exists in pre-aggregate table")
+ intercept[RuntimeException] {
+ sql("alter table preagg1 drop columns(a)").show
+ }.getMessage.contains("cannot be dropped")
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preagg1")
+ }
+
override def afterAll {
sql("DROP TABLE IF EXISTS changedatatypetest")
sql("DROP TABLE IF EXISTS hivetable")
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
index 00e4a14..1a1d5d5 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
@@ -98,6 +98,22 @@ class DropColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
test_drop_and_compaction()
}
+ test("test dropping of column in pre-aggregate should throw exception") {
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preagg1")
+ sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+ sql(
+ "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+ " a,sum(b) from PreAggMain group by a")
+ sql("alter table preaggmain drop columns(c)").show
+ checkExistence(sql("desc table preaggmain"), false, "c")
+ intercept[RuntimeException] {
+ sql("alter table preaggmain drop columns(a)").show
+ }.getMessage.contains("cannot be dropped")
+ sql("drop table if exists preaggMain")
+ sql("drop table if exists preagg1")
+ }
+
override def afterAll {
sql("DROP TABLE IF EXISTS dropcolumntest")
sql("DROP TABLE IF EXISTS hivetable")
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2b5faefa/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
index decb861..c65bcc4 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
@@ -22,12 +22,11 @@ import java.sql.Timestamp
import java.util.Date
import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.test.TestQueryExecutor
import org.scalatest.BeforeAndAfterAll
import org.apache.carbondata.api.CarbonStore
import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.CarbonProperties
class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll {
@@ -142,6 +141,21 @@ class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll {
dropTable(table)
}
+ test("test if delete segments by id is unsupported for pre-aggregate tables") {
+ dropTable("preaggMain")
+ dropTable("preagg1")
+ sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+ sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+ intercept[UnsupportedOperationException] {
+ sql("delete from table preaggMain where segment.id in (1,2)")
+ }.getMessage.contains("Delete segment operation is not supported on tables")
+ intercept[UnsupportedOperationException] {
+ sql("delete from table preagg1 where segment.id in (1,2)")
+ }.getMessage.contains("Delete segment operation is not supported on pre-aggregate tables")
+ dropTable("preaggMain")
+ dropTable("preagg1")
+ }
+
protected def dropTable(tableName: String): Unit ={
sql(s"DROP TABLE IF EXISTS $tableName")
}
[4/9] carbondata git commit: [CARBONDATA-1517]- Pre Aggregate Create
Table Support
Posted by ra...@apache.org.
[CARBONDATA-1517]- Pre Aggregate Create Table Support
Support CTAS in carbon and support creating aggregation tables using CTAS and update aggregation table information to main table schema.
This closes #1433
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/3d1d1ce8
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/3d1d1ce8
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/3d1d1ce8
Branch: refs/heads/master
Commit: 3d1d1ce85a5905f0877ce3e367f255fd71e18ad2
Parents: 3169918
Author: kumarvishal <ku...@gmail.com>
Authored: Sun Oct 15 18:05:55 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530
----------------------------------------------------------------------
.../table/column/ParentColumnTableRelation.java | 71 +++
.../ThriftWrapperSchemaConverterImplTest.java | 28 +-
.../preaggregate/TestPreAggCreateCommand.scala | 148 +++++++
.../carbondata/spark/util/CommonUtil.scala | 9 +
.../spark/sql/catalyst/CarbonDDLSqlParser.scala | 12 +-
.../command/carbonTableSchemaCommon.scala | 170 +++++---
.../command/management/LoadTableCommand.scala | 2 +-
.../CreatePreAggregateTableCommand.scala | 136 ++++++
.../preaaggregate/PreAggregateUtil.scala | 431 +++++++++++++++++++
.../schema/CarbonAlterTableRenameCommand.scala | 2 +-
.../spark/sql/hive/CarbonFileMetastore.scala | 41 +-
.../spark/sql/hive/CarbonHiveMetaStore.scala | 72 +++-
.../apache/spark/sql/hive/CarbonMetaStore.scala | 21 +-
.../sql/parser/CarbonSpark2SqlParser.scala | 2 +-
.../spark/sql/parser/CarbonSparkSqlParser.scala | 33 +-
.../org/apache/spark/util/AlterTableUtil.scala | 10 +-
16 files changed, 1100 insertions(+), 88 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
new file mode 100644
index 0000000..425d0f2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ParentColumnTableRelation.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.metadata.schema.table.column;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.Writable;
+
+/**
+ * To maintain the relation of child column to parent table column
+ */
+public class ParentColumnTableRelation implements Serializable, Writable {
+
+ private RelationIdentifier relationIdentifier;
+ /**
+ * parent column id
+ */
+ private String columnId;
+
+ private String columnName;
+
+ public ParentColumnTableRelation(RelationIdentifier relationIdentifier, String columId,
+ String columnName) {
+ this.relationIdentifier = relationIdentifier;
+ this.columnId = columId;
+ this.columnName = columnName;
+ }
+
+ public RelationIdentifier getRelationIdentifier() {
+ return relationIdentifier;
+ }
+
+ public String getColumnId() {
+ return columnId;
+ }
+
+ public String getColumnName() {
+ return columnName;
+ }
+
+ @Override public void write(DataOutput out) throws IOException {
+ relationIdentifier.write(out);
+ out.writeUTF(columnId);
+ out.writeUTF(columnName);
+ }
+
+ @Override public void readFields(DataInput in) throws IOException {
+ this.relationIdentifier = new RelationIdentifier(null, null, null);
+ relationIdentifier.readFields(in);
+ this.columnId = in.readUTF();
+ this.columnName = in.readUTF();
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
index b8e4eea..d4bb344 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java
@@ -29,6 +29,7 @@ import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
import org.apache.carbondata.core.metadata.schema.table.TableInfo;
import org.apache.carbondata.core.metadata.schema.table.TableSchema;
import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.format.DataMapSchema;
import mockit.Mock;
import mockit.MockUp;
@@ -82,6 +83,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.STRING,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
+ thriftColumnSchema.setAggregate_function("");
thriftColumnSchemas = new ArrayList<org.apache.carbondata.format.ColumnSchema>();
thriftColumnSchemas.add(thriftColumnSchema);
thriftSchemaEvolutionEntries = new ArrayList<>();
@@ -419,6 +421,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.BOOLEAN,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
+ thriftColumnSchema.setAggregate_function("");
ColumnSchema wrapperColumnSchema = new ColumnSchema();
new MockUp<ColumnSchema>() {
@@ -481,6 +484,8 @@ public class ThriftWrapperSchemaConverterImplTest {
@Mock public String getColumnReferenceId() {
return "1";
}
+
+ @Mock public String getAggFunction() {return "" ;}
};
org.apache.carbondata.format.ColumnSchema actualResult =
@@ -494,7 +499,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.STRING,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
-
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
return encodings;
@@ -569,7 +574,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.INT,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
-
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
return encodings;
@@ -643,6 +648,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.SHORT,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
return encodings;
@@ -716,7 +722,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.LONG,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
-
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
return encodings;
@@ -790,6 +796,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.DOUBLE,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
@@ -864,6 +871,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.DECIMAL,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
return encodings;
@@ -924,6 +932,10 @@ public class ThriftWrapperSchemaConverterImplTest {
@Mock public String getColumnReferenceId() {
return "1";
}
+
+ @Mock public String getAggFunction() {
+ return "";
+ }
};
ColumnSchema wrapperColumnSchema = new ColumnSchema();
@@ -938,6 +950,7 @@ public class ThriftWrapperSchemaConverterImplTest {
org.apache.carbondata.format.DataType.TIMESTAMP, "columnName", "1", true, encoders,
true);
thriftColumnSchema.setSchemaOrdinal(1);
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
@@ -1012,7 +1025,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.ARRAY,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
-
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
return encodings;
@@ -1086,6 +1099,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.STRUCT,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
@@ -1166,6 +1180,7 @@ public class ThriftWrapperSchemaConverterImplTest {
encoders,
true);
thriftColumnSchema.setSchemaOrdinal(1);
+ thriftColumnSchema.setAggregate_function("");
new MockUp<ColumnSchema>() {
@Mock public List<Encoding> getEncodingList() {
@@ -1316,7 +1331,7 @@ public class ThriftWrapperSchemaConverterImplTest {
new org.apache.carbondata.format.ColumnSchema(org.apache.carbondata.format.DataType.BOOLEAN,
"columnName", "1", true, encoders, true);
thriftColumnSchema.setSchemaOrdinal(1);
-
+ thriftColumnSchema.setAggregate_function("");
ColumnSchema wrapperColumnSchema = new ColumnSchema();
org.apache.carbondata.format.ColumnSchema actualResult =
thriftWrapperSchemaConverter.fromWrapperToExternalColumnSchema(wrapperColumnSchema);
@@ -1504,6 +1519,8 @@ public class ThriftWrapperSchemaConverterImplTest {
@Mock public String getColumnReferenceId() {
return "1";
}
+
+ @Mock public String getAggFunction() { return "";}
};
new MockUp<TableInfo>() {
@@ -1540,6 +1557,7 @@ public class ThriftWrapperSchemaConverterImplTest {
org.apache.carbondata.format.TableInfo expectedResult =
new org.apache.carbondata.format.TableInfo(thriftFactTable, new ArrayList<org.apache
.carbondata.format.TableSchema>());
+ expectedResult.setDataMapSchemas(new ArrayList<DataMapSchema>());
assertEquals(expectedResult, actualResult);
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
new file mode 100644
index 0000000..6120e88
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
@@ -0,0 +1,148 @@
+package org.apache.carbondata.integration.spark.testsuite.preaggregate
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
+
+ override def beforeAll {
+ sql("drop table if exists PreAggMain")
+ sql("drop table if exists PreAggMain1")
+ sql("drop table if exists PreAggMain2")
+ sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
+ sql("create table preaggMain1 (a string, b string, c string) stored by 'carbondata' tblProperties('DICTIONARY_INCLUDE' = 'a')")
+ sql("create table preaggMain2 (a string, b string, c string) stored by 'carbondata'")
+ }
+
+
+ test("test pre agg create table One") {
+ sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg1"), true, "preaggmain_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg1"), true, "preaggmain_b_sum")
+ sql("drop table preagg1")
+ }
+
+ test("test pre agg create table Two") {
+ sql("create table preagg2 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a as a1,sum(b) from PreAggMain group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg2"), true, "preaggmain_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg2"), true, "preaggmain_b_sum")
+ sql("drop table preagg2")
+ }
+
+ test("test pre agg create table Three") {
+ sql("create table preagg3 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) as sum from PreAggMain group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg3"), true, "preaggmain_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg3"), true, "preaggmain_b_sum")
+ sql("drop table preagg3")
+ }
+
+ test("test pre agg create table four") {
+ sql("create table preagg4 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a as a1,sum(b) as sum from PreAggMain group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg4"), true, "preaggmain_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg4"), true, "preaggmain_b_sum")
+ sql("drop table preagg4")
+ }
+
+
+ test("test pre agg create table five") {
+ sql("create table preagg11 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a,sum(b) from PreAggMain1 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "preaggmain1_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "preaggmain1_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "DICTIONARY")
+ sql("drop table preagg11")
+ }
+
+ test("test pre agg create table six") {
+ sql("create table preagg12 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a as a1,sum(b) from PreAggMain1 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "preaggmain1_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "preaggmain1_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "DICTIONARY")
+ sql("drop table preagg12")
+ }
+
+ test("test pre agg create table seven") {
+ sql("create table preagg13 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a,sum(b) as sum from PreAggMain1 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "preaggmain1_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "preaggmain1_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "DICTIONARY")
+ sql("drop table preagg13")
+ }
+
+ test("test pre agg create table eight") {
+ sql("create table preagg14 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a as a1,sum(b) as sum from PreAggMain1 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "preaggmain1_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "preaggmain1_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "DICTIONARY")
+ sql("drop table preagg14")
+ }
+
+
+ test("test pre agg create table nine") {
+ sql("create table preagg15 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a,avg(b) from PreAggMain2 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_b_count")
+ sql("drop table preagg15")
+ }
+
+ test("test pre agg create table ten") {
+ sql("create table preagg16 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,max(b) from PreAggMain2 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg16"), true, "preaggmain2_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg16"), true, "preaggmain2_b_max")
+ sql("drop table preagg16")
+ }
+
+ test("test pre agg create table eleven") {
+ sql("create table preagg17 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a,min(b) from PreAggMain2 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg17"), true, "preaggmain2_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg17"), true, "preaggmain2_b_min")
+ sql("drop table preagg17")
+ }
+
+ test("test pre agg create table twelve") {
+ sql("create table preagg18 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,count(b) from PreAggMain2 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg18"), true, "preaggmain2_a")
+ checkExistence(sql("DESCRIBE FORMATTED preagg18"), true, "preaggmain2_b_count")
+ sql("drop table preagg18")
+ }
+
+ test("test pre agg create table thirteen") {
+ try {
+ sql(
+ "create table preagg19 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,count(distinct b) from PreAggMain2 group by a")
+ assert(false)
+ } catch {
+ case _: Exception =>
+ assert(true)
+ }
+ }
+
+ test("test pre agg create table fourteen") {
+ try {
+ sql(
+ "create table preagg20 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,sum(distinct b) from PreAggMain2 group by a")
+ assert(false)
+ } catch {
+ case _: Exception =>
+ assert(true)
+ }
+ }
+
+ test("test pre agg create table fifteen") {
+ try {
+ sql(
+ "create table preagg21 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,sum(b) from PreAggMain2 where a='vishal' group by a")
+ assert(false)
+ } catch {
+ case _: Exception =>
+ assert(true)
+ }
+ }
+
+
+ override def afterAll {
+ sql("drop table if exists PreAggMain")
+ sql("drop table if exists PreAggMain1")
+ sql("drop table if exists PreAggMain2")
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 27ebf42..84294ff 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -20,6 +20,7 @@ package org.apache.carbondata.spark.util
import java.text.SimpleDateFormat
import java.util
+import java.util.regex.{Matcher, Pattern}
import scala.collection.JavaConverters._
import scala.collection.mutable.Map
@@ -834,6 +835,14 @@ object CommonUtil {
}
}
+ def getScaleAndPrecision(dataType: String): (Int, Int) = {
+ val m: Matcher = Pattern.compile("^decimal\\(([^)]+)\\)").matcher(dataType)
+ m.find()
+ val matchedString: String = m.group(1)
+ val scaleAndPrecision = matchedString.split(",")
+ (Integer.parseInt(scaleAndPrecision(0).trim), Integer.parseInt(scaleAndPrecision(1).trim))
+ }
+
/**
* Merge the carbonindex files with in the segment to carbonindexmerge file inside same segment
*/
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index ee51954..bb80bce 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -221,16 +221,6 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
dimensions ++ complexDimensions
}
-
-
- def getScaleAndPrecision(dataType: String): (Int, Int) = {
- val m: Matcher = Pattern.compile("^decimal\\(([^)]+)\\)").matcher(dataType)
- m.find()
- val matchedString: String = m.group(1)
- val scaleAndPrecision = matchedString.split(",")
- (Integer.parseInt(scaleAndPrecision(0).trim), Integer.parseInt(scaleAndPrecision(1).trim))
- }
-
/**
* This will prepate the Model from the Tree details.
*
@@ -1074,7 +1064,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
// if it is present then extracting the precision and scale. resetting the data type
// with Decimal.
case _ if dataType.startsWith("decimal") =>
- val (precision, scale) = getScaleAndPrecision(dataType)
+ val (precision, scale) = CommonUtil.getScaleAndPrecision(dataType)
Field(field.column,
Some("Decimal"),
field.name,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 1188b59..37ba8a5 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -33,8 +33,8 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier
import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes, DecimalType}
import org.apache.carbondata.core.metadata.encoder.Encoding
import org.apache.carbondata.core.metadata.schema._
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier, TableInfo, TableSchema}
+import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema, ParentColumnTableRelation}
import org.apache.carbondata.core.service.CarbonCommonFactory
import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentUpdateStatusManager}
import org.apache.carbondata.core.util.DataTypeUtil
@@ -61,13 +61,30 @@ case class TableModel(
util.List[ColumnProperty]]] = None,
bucketFields: Option[BucketFields],
partitionInfo: Option[PartitionInfo],
- tableComment: Option[String] = None)
+ tableComment: Option[String] = None,
+ var parentTable: Option[CarbonTable] = None,
+ var dataMapRelation: Option[scala.collection.mutable.LinkedHashMap[Field, DataMapField]] = None)
case class Field(column: String, var dataType: Option[String], name: Option[String],
children: Option[List[Field]], parent: String = null,
storeType: Option[String] = Some("columnar"),
var schemaOrdinal: Int = -1,
- var precision: Int = 0, var scale: Int = 0, var rawSchema: String = "")
+ var precision: Int = 0, var scale: Int = 0, var rawSchema: String = "") {
+ override def equals(o: Any) : Boolean = o match {
+ case that: Field =>
+ that.column.equalsIgnoreCase(this.column)
+ case _ => false
+ }
+ override def hashCode : Int = column.hashCode
+}
+
+case class DataMapField(aggregateFunction: String = "",
+ columnTableRelation: Option[ColumnTableRelation] = None) {
+}
+
+case class ColumnTableRelation(parentColumnName: String, parentColumnId: String,
+ parentTableName: String, parentDatabaseName: String, parentTableId: String) {
+}
case class ColumnProperty(key: String, value: String)
@@ -358,15 +375,13 @@ class TableNewProcessor(cm: TableModel) {
fields.foreach(field => {
val encoders = new java.util.ArrayList[Encoding]()
encoders.add(Encoding.DICTIONARY)
- val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
- field,
+ val columnSchema: ColumnSchema = getColumnSchema(
+ DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
+ field.name.getOrElse(field.column),
encoders,
- isDimensionCol = true,
- field.precision,
- field.scale,
- field.schemaOrdinal,
- cm.highcardinalitydims.getOrElse(Seq()),
- cm.databaseName)
+ true,
+ field,
+ cm.dataMapRelation)
allColumns ++= Seq(columnSchema)
if (field.children.get != null) {
columnSchema.setNumberOfChild(field.children.get.size)
@@ -377,6 +392,56 @@ class TableNewProcessor(cm: TableModel) {
allColumns
}
+ def getColumnSchema(
+ dataType: DataType,
+ colName: String,
+ encoders: java.util.List[Encoding],
+ isDimensionCol: Boolean,
+ field: Field,
+ map: Option[scala.collection.mutable.LinkedHashMap[Field, DataMapField]]) : ColumnSchema = {
+ val columnSchema = new ColumnSchema()
+ columnSchema.setDataType(dataType)
+ columnSchema.setColumnName(colName)
+ val isParentColumnRelation = map.isDefined && map.get.get(field).isDefined
+ if(!isParentColumnRelation) {
+ val highCardinalityDims = cm.highcardinalitydims.getOrElse(Seq())
+ if (highCardinalityDims.contains(colName)) {
+ encoders.remove(Encoding.DICTIONARY)
+ }
+ if (dataType == DataTypes.DATE) {
+ encoders.add(Encoding.DIRECT_DICTIONARY)
+ }
+ if (dataType == DataTypes.TIMESTAMP && !highCardinalityDims.contains(colName)) {
+ encoders.add(Encoding.DIRECT_DICTIONARY)
+ }
+ }
+ columnSchema.setEncodingList(encoders)
+ val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
+ val columnUniqueId = colUniqueIdGenerator.generateUniqueId(cm.databaseName,
+ columnSchema)
+ columnSchema.setColumnUniqueId(columnUniqueId)
+ columnSchema.setColumnReferenceId(columnUniqueId)
+ columnSchema.setDimensionColumn(isDimensionCol)
+ columnSchema.setPrecision(field.precision)
+ columnSchema.setScale(field.scale)
+ columnSchema.setSchemaOrdinal(field.schemaOrdinal)
+ columnSchema.setSortColumn(false)
+ if(isParentColumnRelation) {
+ val dataMapField = map.get.get(field).get
+ columnSchema.setAggFunction(dataMapField.aggregateFunction);
+ val relation = dataMapField.columnTableRelation.get
+ val parentColumnTableRelationList = new util.ArrayList[ParentColumnTableRelation]
+ val relationIdentifier = new RelationIdentifier(
+ relation.parentDatabaseName, relation.parentTableName, relation.parentTableId)
+ val parentColumnTableRelation = new ParentColumnTableRelation(
+ relationIdentifier, relation.parentColumnId, relation.parentColumnName)
+ parentColumnTableRelationList.add(parentColumnTableRelation)
+ columnSchema.setParentColumnTableRelations(parentColumnTableRelationList)
+ }
+ // TODO: Need to fill RowGroupID, converted type
+ // & Number of Children after DDL finalization
+ columnSchema
+ }
// process create dml fields and create wrapper TableInfo object
def process: TableInfo = {
@@ -388,17 +453,22 @@ class TableNewProcessor(cm: TableModel) {
// Sort columns should be at the begin of all columns
cm.sortKeyDims.get.foreach { keyDim =>
val field = cm.dimCols.find(keyDim equals _.column).get
- val encoders = new java.util.ArrayList[Encoding]()
- encoders.add(Encoding.DICTIONARY)
- val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
- field,
+ val encoders = if (cm.parentTable.isDefined && cm.dataMapRelation.get.get(field).isDefined) {
+ cm.parentTable.get.getColumnByName(
+ cm.parentTable.get.getFactTableName,
+ cm.dataMapRelation.get.get(field).get.columnTableRelation.get.parentColumnName).getEncoder
+ } else {
+ val encoders = new java.util.ArrayList[Encoding]()
+ encoders.add(Encoding.DICTIONARY)
+ encoders
+ }
+ val columnSchema = getColumnSchema(
+ DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
+ field.name.getOrElse(field.column),
encoders,
- isDimensionCol = true,
- field.precision,
- field.scale,
- field.schemaOrdinal,
- cm.highcardinalitydims.getOrElse(Seq()),
- cm.databaseName)
+ true,
+ field,
+ cm.dataMapRelation)
columnSchema.setSortColumn(true)
allColumns :+= columnSchema
index = index + 1
@@ -407,17 +477,24 @@ class TableNewProcessor(cm: TableModel) {
cm.dimCols.foreach { field =>
val sortField = cm.sortKeyDims.get.find(field.column equals _)
if (sortField.isEmpty) {
- val encoders = new java.util.ArrayList[Encoding]()
- encoders.add(Encoding.DICTIONARY)
- val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
- field,
+ val encoders = if (cm.parentTable.isDefined &&
+ cm.dataMapRelation.get.get(field).isDefined) {
+ cm.parentTable.get.getColumnByName(
+ cm.parentTable.get.getFactTableName,
+ cm.dataMapRelation.get.get(field).get.
+ columnTableRelation.get.parentColumnName).getEncoder
+ } else {
+ val encoders = new java.util.ArrayList[Encoding]()
+ encoders.add(Encoding.DICTIONARY)
+ encoders
+ }
+ val columnSchema = getColumnSchema(
+ DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
+ field.name.getOrElse(field.column),
encoders,
- isDimensionCol = true,
- field.precision,
- field.scale,
- field.schemaOrdinal,
- cm.highcardinalitydims.getOrElse(Seq()),
- cm.databaseName)
+ true,
+ field,
+ cm.dataMapRelation)
allColumns :+= columnSchema
index = index + 1
if (field.children.isDefined && field.children.get != null) {
@@ -429,15 +506,13 @@ class TableNewProcessor(cm: TableModel) {
cm.msrCols.foreach { field =>
val encoders = new java.util.ArrayList[Encoding]()
- val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
- field,
+ val columnSchema = getColumnSchema(
+ DataTypeConverterUtil.convertToCarbonType(field.dataType.getOrElse("")),
+ field.name.getOrElse(field.column),
encoders,
- isDimensionCol = false,
- field.precision,
- field.scale,
- field.schemaOrdinal,
- cm.highcardinalitydims.getOrElse(Seq()),
- cm.databaseName)
+ false,
+ field,
+ cm.dataMapRelation)
allColumns :+= columnSchema
index = index + 1
measureCount += 1
@@ -486,15 +561,13 @@ class TableNewProcessor(cm: TableModel) {
Some(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE),
None
)
- val columnSchema: ColumnSchema = TableNewProcessor.createColumnSchema(
- field,
+ val columnSchema: ColumnSchema = getColumnSchema(
+ DataTypes.DOUBLE,
+ CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE,
encoders,
- isDimensionCol = false,
- field.precision,
- field.scale,
- -1,
- cm.highcardinalitydims.getOrElse(Seq()),
- cm.databaseName)
+ false,
+ field,
+ cm.dataMapRelation)
columnSchema.setInvisible(true)
allColumns :+= columnSchema
}
@@ -503,6 +576,7 @@ class TableNewProcessor(cm: TableModel) {
val tableInfo = new TableInfo()
val tableSchema = new TableSchema()
+
val schemaEvol = new SchemaEvolution()
schemaEvol.setSchemaEvolutionEntryList(new util.ArrayList[SchemaEvolutionEntry]())
tableSchema.setTableId(UUID.randomUUID().toString)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/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
index bda6829..222c30d 100644
--- 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
@@ -364,7 +364,7 @@ case class LoadTableCommand(
entry.setTime_stamp(System.currentTimeMillis())
// write TableInfo
- metastore.updateTableSchema(carbonTablePath.getCarbonTableIdentifier,
+ metastore.updateTableSchemaForAlter(carbonTablePath.getCarbonTableIdentifier,
carbonTablePath.getCarbonTableIdentifier,
tableInfo, entry, carbonTablePath.getPath)(sparkSession)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
new file mode 100644
index 0000000..ca384f9
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -0,0 +1,136 @@
+/*
+ * 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.preaaggregate
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+
+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.{RelationIdentifier, TableInfo}
+import org.apache.carbondata.core.util.CarbonUtil
+
+/**
+ * Below command class will be used to create pre-aggregate table
+ * and updating the parent table about the child table information
+ * Failure case:
+ * 1. failed to create pre aggregate table.
+ * 2. failed to update main table
+ *
+ * @param cm
+ * @param dataFrame
+ * @param createDSTable
+ * @param queryString
+ */
+case class CreatePreAggregateTableCommand(
+ cm: TableModel,
+ dataFrame: DataFrame,
+ createDSTable: Boolean = true,
+ queryString: String,
+ fieldRelationMap: scala.collection.mutable.LinkedHashMap[Field, DataMapField])
+ 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]")
+ // getting the parent table
+ val parentTable = PreAggregateUtil.getParentCarbonTable(dataFrame.logicalPlan)
+ // getting the table name
+ val parentTableName = parentTable.getFactTableName
+ // getting the db name of parent table
+ val parentDbName = parentTable.getDatabaseName
+ // updating the relation identifier, this will be stored in child table
+ // which can be used during dropping of pre-aggreate table as parent table will
+ // also get updated
+ cm.parentTable = Some(parentTable)
+ cm.dataMapRelation = Some(fieldRelationMap)
+ 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) """)
+ // child schema object which will be updated on parent table about the
+ val childSchema = tableInfo.getFactTable
+ .buildChildSchema("", tableInfo.getDatabaseName, queryString, "AGGREGATION")
+ // upadting the parent table about child table
+ PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)
+ } 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/3d1d1ce8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
new file mode 100644
index 0000000..c4b6783
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -0,0 +1,431 @@
+/*
+ * 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.preaaggregate
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast}
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.execution.command.{ColumnTableRelation, DataMapField, Field}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.sql.hive.HiveExternalCatalog.{DATASOURCE_SCHEMA_NUMPARTS, DATASOURCE_SCHEMA_PART_PREFIX}
+import org.apache.spark.sql.types.DataType
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.TableInfo
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CommonUtil
+
+/**
+ * Utility class for keeping all the utility method for pre-aggregate
+ */
+object PreAggregateUtil {
+
+ private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+ def getParentCarbonTable(plan: LogicalPlan): CarbonTable = {
+ plan match {
+ case Aggregate(_, aExp, SubqueryAlias(_, l: LogicalRelation, _))
+ if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+ l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation.metaData.carbonTable
+ case _ => throw new MalformedCarbonCommandException("table does not exist")
+ }
+ }
+
+ /**
+ * Below method will be used to validate the select plan
+ * and get the required fields from select plan
+ * Currently only aggregate query is support any other type of query will
+ * fail
+ * @param plan
+ * @param selectStmt
+ * @return list of fields
+ */
+ def validateActualSelectPlanAndGetAttrubites(plan: LogicalPlan,
+ selectStmt: String): scala.collection.mutable.LinkedHashMap[Field, DataMapField] = {
+ val fieldToDataMapFieldMap = scala.collection.mutable.LinkedHashMap.empty[Field, DataMapField]
+ plan match {
+ case Aggregate(_, aExp, SubqueryAlias(_, l: LogicalRelation, _))
+ if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+ val carbonTable = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonRelation
+ .metaData.carbonTable
+ val parentTableName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+ .getTableName
+ val parentDatabaseName = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+ .getDatabaseName
+ val parentTableId = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
+ .getTableId
+ if (!carbonTable.getTableInfo.getParentRelationIdentifiers.isEmpty) {
+ throw new MalformedCarbonCommandException(
+ "Pre Aggregation is not supported on Pre-Aggregated Table")
+ }
+ aExp.map {
+ case Alias(attr: AggregateExpression, _) =>
+ if (attr.isDistinct) {
+ throw new MalformedCarbonCommandException(
+ "Distinct is not supported On Pre Aggregation")
+ }
+ fieldToDataMapFieldMap ++= ((validateAggregateFunctionAndGetFields(carbonTable,
+ attr.aggregateFunction,
+ parentTableName,
+ parentDatabaseName,
+ parentTableId)))
+ case attr: AttributeReference =>
+ fieldToDataMapFieldMap += getField(attr.name,
+ attr.dataType,
+ parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName = parentTableName,
+ parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+ case Alias(attr: AttributeReference, _) =>
+ fieldToDataMapFieldMap += getField(attr.name,
+ attr.dataType,
+ parentColumnId = carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName = parentTableName,
+ parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+ case _ =>
+ throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${
+ selectStmt } ")
+ }
+ Some(carbonTable)
+ case _ =>
+ throw new MalformedCarbonCommandException(s"Unsupported Select Statement:${ selectStmt } ")
+ }
+ fieldToDataMapFieldMap
+ }
+
+ /**
+ * Below method will be used to validate about the aggregate function
+ * which is applied on select query.
+ * Currently sum, max, min, count, avg is supported
+ * in case of any other aggregate function it will throw error
+ * In case of avg it will return two fields one for count
+ * and other of sum of that column to support rollup
+ * @param carbonTable
+ * @param aggFunctions
+ * @param parentTableName
+ * @param parentDatabaseName
+ * @param parentTableId
+ * @return list of fields
+ */
+ def validateAggregateFunctionAndGetFields(carbonTable: CarbonTable,
+ aggFunctions: AggregateFunction,
+ parentTableName: String,
+ parentDatabaseName: String,
+ parentTableId: String) : scala.collection.mutable.ListBuffer[(Field, DataMapField)] = {
+ val list = scala.collection.mutable.ListBuffer.empty[(Field, DataMapField)]
+ aggFunctions match {
+ case sum@Sum(attr: AttributeReference) =>
+ list += getField(attr.name,
+ attr.dataType,
+ sum.prettyName,
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case sum@Sum(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ list += getField(attr.name,
+ changeDataType,
+ sum.prettyName,
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case count@Count(Seq(attr: AttributeReference)) =>
+ list += getField(attr.name,
+ attr.dataType,
+ count.prettyName,
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case min@Min(attr: AttributeReference) =>
+ list += getField(attr.name,
+ attr.dataType,
+ min.prettyName,
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case min@Min(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ list += getField(attr.name,
+ changeDataType,
+ min.prettyName,
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case max@Max(attr: AttributeReference) =>
+ list += getField(attr.name,
+ attr.dataType,
+ max.prettyName,
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case max@Max(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ list += getField(attr.name,
+ changeDataType,
+ max.prettyName,
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case Average(attr: AttributeReference) =>
+ getField(attr.name,
+ attr.dataType,
+ "sum",
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ list += getField(attr.name,
+ attr.dataType,
+ "count",
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case Average(Cast(attr: AttributeReference, changeDataType: DataType)) =>
+ list += getField(attr.name,
+ changeDataType,
+ "sum",
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ list += getField(attr.name,
+ changeDataType,
+ "count",
+ carbonTable.getColumnByName(parentTableName, attr.name).getColumnId,
+ parentTableName,
+ parentDatabaseName, parentTableId = parentTableId)
+ case _ =>
+ throw new MalformedCarbonCommandException("Un-Supported Aggregation Type")
+ }
+ }
+
+ /**
+ * Below method will be used to get the fields object for pre aggregate table
+ * @param columnName
+ * @param dataType
+ * @param aggregateType
+ * @param parentColumnId
+ * @param parentTableName
+ * @param parentDatabaseName
+ * @param parentTableId
+ * @return fields object
+ */
+ def getField(columnName: String,
+ dataType: DataType,
+ aggregateType: String = "",
+ parentColumnId: String,
+ parentTableName: String,
+ parentDatabaseName: String,
+ parentTableId: String): (Field, DataMapField) = {
+ val actualColumnName = if (aggregateType.equals("")) {
+ parentTableName + '_' + columnName
+ } else {
+ parentTableName + '_' + columnName + '_' + aggregateType
+ }
+ val rawSchema = '`' + actualColumnName + '`' + ' ' + dataType.typeName
+ val columnTableRelation = ColumnTableRelation(parentColumnName = columnName,
+ parentColumnId = parentColumnId,
+ parentTableName = parentTableName,
+ parentDatabaseName = parentDatabaseName, parentTableId = parentTableId)
+ val dataMapField = DataMapField(aggregateType, Some(columnTableRelation))
+ if (dataType.typeName.startsWith("decimal")) {
+ val (precision, scale) = CommonUtil.getScaleAndPrecision(dataType.catalogString)
+ (Field(column = actualColumnName,
+ dataType = Some(dataType.typeName),
+ name = Some(actualColumnName),
+ children = None,
+ precision = precision,
+ scale = scale,
+ rawSchema = rawSchema), dataMapField)
+ }
+ else {
+ (Field(column = actualColumnName,
+ dataType = Some(dataType.typeName),
+ name = Some(actualColumnName),
+ children = None,
+ rawSchema = rawSchema), dataMapField)
+ }
+ }
+
+ /**
+ * Below method will be used to update the main table about the pre aggregate table information
+ * in case of any exption it will throw error so pre aggregate table creation will fail
+ * @param dbName
+ * @param tableName
+ * @param childSchema
+ * @param sparkSession
+ */
+ def updateMainTable(dbName: String, tableName: String,
+ childSchema: DataMapSchema, sparkSession: SparkSession): Unit = {
+ val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+ val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+ LockUsage.DROP_TABLE_LOCK)
+ var locks = List.empty[ICarbonLock]
+ var carbonTable: CarbonTable = null
+ var numberOfCurrentChild: Int = 0
+ try {
+ val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+ carbonTable = metastore
+ .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+ .tableMeta.carbonTable
+ locks = acquireLock(dbName, tableName, locksToBeAcquired, 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)
+ numberOfCurrentChild = wrapperTableInfo.getDataMapSchemaList.size
+ wrapperTableInfo.getDataMapSchemaList.add(childSchema)
+ val thriftTable = schemaConverter
+ .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+ updateSchemaInfo(carbonTable,
+ thriftTable)(sparkSession,
+ sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+ LOGGER.info(s"Pre Aggeragte Parent table updated is successful for table $dbName.$tableName")
+ } catch {
+ case e: Exception =>
+ LOGGER.error(e, "Pre Aggregate Parent table update failed reverting changes")
+ revertMainTableChanges(dbName, tableName, numberOfCurrentChild)(sparkSession)
+ throw e
+ } finally {
+ // release lock after command execution completion
+ releaseLocks(locks)
+ }
+ Seq.empty
+ }
+
+ /**
+ * Below method will be used to update the main table schema
+ * @param carbonTable
+ * @param thriftTable
+ * @param sparkSession
+ * @param sessionState
+ */
+ def updateSchemaInfo(carbonTable: CarbonTable,
+ thriftTable: TableInfo)(sparkSession: SparkSession,
+ sessionState: CarbonSessionState): Unit = {
+ val dbName = carbonTable.getDatabaseName
+ val tableName = carbonTable.getFactTableName
+ CarbonEnv.getInstance(sparkSession).carbonMetastore
+ .updateTableSchemaForDataMap(carbonTable.getCarbonTableIdentifier,
+ carbonTable.getCarbonTableIdentifier,
+ thriftTable,
+ carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+ val tableIdentifier = TableIdentifier(tableName, Some(dbName))
+ sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
+ }
+
+ /**
+ * This method will split schema string into multiple parts of configured size and
+ * registers the parts as keys in tableProperties which will be read by spark to prepare
+ * Carbon Table fields
+ *
+ * @param sparkConf
+ * @param schemaJsonString
+ * @return
+ */
+ private def prepareSchemaJson(sparkConf: SparkConf,
+ schemaJsonString: String): String = {
+ val threshold = sparkConf
+ .getInt(CarbonCommonConstants.SPARK_SCHEMA_STRING_LENGTH_THRESHOLD,
+ CarbonCommonConstants.SPARK_SCHEMA_STRING_LENGTH_THRESHOLD_DEFAULT)
+ // Split the JSON string.
+ val parts = schemaJsonString.grouped(threshold).toSeq
+ var schemaParts: Seq[String] = Seq.empty
+ schemaParts = schemaParts :+ s"'$DATASOURCE_SCHEMA_NUMPARTS'='${ parts.size }'"
+ parts.zipWithIndex.foreach { case (part, index) =>
+ schemaParts = schemaParts :+ s"'$DATASOURCE_SCHEMA_PART_PREFIX$index'='$part'"
+ }
+ schemaParts.mkString(",")
+ }
+
+ /**
+ * Validates that the table exists and acquires meta lock on it.
+ *
+ * @param dbName
+ * @param tableName
+ * @return
+ */
+ def acquireLock(dbName: String,
+ tableName: String,
+ locksToBeAcquired: List[String],
+ table: CarbonTable): List[ICarbonLock] = {
+ // acquire the lock first
+ val acquiredLocks = ListBuffer[ICarbonLock]()
+ try {
+ locksToBeAcquired.foreach { lock =>
+ acquiredLocks += CarbonLockUtil.getLockObject(table.getCarbonTableIdentifier, lock)
+ }
+ acquiredLocks.toList
+ } catch {
+ case e: Exception =>
+ releaseLocks(acquiredLocks.toList)
+ throw e
+ }
+ }
+
+ /**
+ * This method will release the locks acquired for an operation
+ *
+ * @param locks
+ */
+ def releaseLocks(locks: List[ICarbonLock]): Unit = {
+ locks.foreach { carbonLock =>
+ if (carbonLock.unlock()) {
+ LOGGER.info("Pre agg table lock released successfully")
+ } else {
+ LOGGER.error("Unable to release lock during Pre agg table cretion")
+ }
+ }
+ }
+
+ /**
+ * This method reverts the changes to the schema if add column command fails.
+ *
+ * @param dbName
+ * @param tableName
+ * @param numberOfChildSchema
+ * @param sparkSession
+ */
+ def revertMainTableChanges(dbName: String, tableName: String, numberOfChildSchema: Int)
+ (sparkSession: SparkSession): Unit = {
+ val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+ val carbonTable = metastore
+ .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation].tableMeta
+ .carbonTable
+ carbonTable.getTableLastUpdatedTime
+ val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+ carbonTable.getCarbonTableIdentifier)
+ val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+ if (thriftTable.dataMapSchemas.size > numberOfChildSchema) {
+ metastore
+ .revertTableSchemaForPreAggCreationFailure(carbonTable.getCarbonTableIdentifier,
+ thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index 5936355..e0617d6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -109,7 +109,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
}
val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
- val newTablePath = metastore.updateTableSchema(newTableIdentifier,
+ val newTablePath = metastore.updateTableSchemaForAlter(newTableIdentifier,
carbonTable.getCarbonTableIdentifier,
tableInfo,
schemaEvolutionEntry,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index 9822d8f..51c7f3b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -43,6 +43,7 @@ import org.apache.carbondata.core.util.CarbonUtil
import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
import org.apache.carbondata.core.writer.ThriftWriter
import org.apache.carbondata.events.{LookupRelationPostEvent, OperationContext, OperationListenerBus}
+import org.apache.carbondata.format
import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
import org.apache.carbondata.processing.merger.TableMeta
import org.apache.carbondata.spark.util.CarbonSparkUtil
@@ -111,6 +112,22 @@ class CarbonFileMetastore extends CarbonMetaStore {
}
}
+ /**
+ * This method will overwrite the existing schema and update it with the given details
+ *
+ * @param newTableIdentifier
+ * @param thriftTableInfo
+ * @param carbonStorePath
+ * @param sparkSession
+ */
+ def updateTableSchemaForDataMap(newTableIdentifier: CarbonTableIdentifier,
+ oldTableIdentifier: CarbonTableIdentifier,
+ thriftTableInfo: org.apache.carbondata.format.TableInfo,
+ carbonStorePath: String)(sparkSession: SparkSession): String = {
+ updateTableSchemaForAlter(newTableIdentifier,
+ oldTableIdentifier, thriftTableInfo, null, carbonStorePath) (sparkSession)
+ }
+
def lookupRelation(dbName: Option[String], tableName: String)
(sparkSession: SparkSession): LogicalPlan = {
lookupRelation(TableIdentifier(tableName, dbName))(sparkSession)
@@ -214,7 +231,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
* @param tablePath
* @param sparkSession
*/
- def updateTableSchema(newTableIdentifier: CarbonTableIdentifier,
+ def updateTableSchemaForAlter(newTableIdentifier: CarbonTableIdentifier,
oldTableIdentifier: CarbonTableIdentifier,
thriftTableInfo: org.apache.carbondata.format.TableInfo,
schemaEvolutionEntry: SchemaEvolutionEntry,
@@ -251,7 +268,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
* @param tablePath
* @param sparkSession
*/
- def revertTableSchema(carbonTableIdentifier: CarbonTableIdentifier,
+ def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
thriftTableInfo: org.apache.carbondata.format.TableInfo,
tablePath: String)(sparkSession: SparkSession): String = {
val tableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
@@ -271,7 +288,27 @@ class CarbonFileMetastore extends CarbonMetaStore {
path
}
+ override def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier:
+ CarbonTableIdentifier,
+ thriftTableInfo: org.apache.carbondata.format.TableInfo,
+ tablePath: String)(sparkSession: SparkSession): String = {
+ val tableIdentifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+ val schemaConverter = new ThriftWrapperSchemaConverterImpl
+ val wrapperTableInfo = schemaConverter
+ .fromExternalToWrapperTableInfo(thriftTableInfo,
+ carbonTableIdentifier.getDatabaseName,
+ carbonTableIdentifier.getTableName,
+ tableIdentifier.getStorePath)
+ val childSchemaList = wrapperTableInfo.getDataMapSchemaList
+ childSchemaList.remove(childSchemaList.size() - 1)
+ wrapperTableInfo.setStorePath(tableIdentifier.getStorePath)
+ val path = createSchemaThriftFile(wrapperTableInfo,
+ thriftTableInfo,
+ tableIdentifier.getCarbonTableIdentifier)
+ addTableCache(wrapperTableInfo, tableIdentifier)
+ path
+ }
/**
*
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
index 76241a6..c64b7bb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
@@ -115,7 +115,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
* @param schemaEvolutionEntry
* @param sparkSession
*/
- override def updateTableSchema(newTableIdentifier: CarbonTableIdentifier,
+ override def updateTableSchemaForAlter(newTableIdentifier: CarbonTableIdentifier,
oldTableIdentifier: CarbonTableIdentifier,
thriftTableInfo: format.TableInfo,
schemaEvolutionEntry: SchemaEvolutionEntry,
@@ -126,7 +126,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
if (schemaEvolutionEntry != null) {
thriftTableInfo.fact_table.schema_evolution.schema_evolution_history.add(schemaEvolutionEntry)
}
- updateHiveMetaStore(newTableIdentifier,
+ updateHiveMetaStoreForAlter(newTableIdentifier,
oldTableIdentifier,
thriftTableInfo,
identifier.getStorePath,
@@ -134,7 +134,29 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
schemaConverter)
}
- private def updateHiveMetaStore(newTableIdentifier: CarbonTableIdentifier,
+ /**
+ * This method will overwrite the existing schema and update it with the given details
+ *
+ * @param newTableIdentifier
+ * @param thriftTableInfo
+ * @param carbonStorePath
+ * @param sparkSession
+ */
+ override def updateTableSchemaForDataMap(newTableIdentifier: CarbonTableIdentifier,
+ oldTableIdentifier: CarbonTableIdentifier,
+ thriftTableInfo: org.apache.carbondata.format.TableInfo,
+ carbonStorePath: String)(sparkSession: SparkSession): String = {
+ val schemaConverter = new ThriftWrapperSchemaConverterImpl
+ val identifier = AbsoluteTableIdentifier.fromTablePath(carbonStorePath)
+ updateHiveMetaStoreForDataMap(newTableIdentifier,
+ oldTableIdentifier,
+ thriftTableInfo,
+ identifier.getStorePath,
+ sparkSession,
+ schemaConverter)
+ }
+
+ private def updateHiveMetaStoreForAlter(newTableIdentifier: CarbonTableIdentifier,
oldTableIdentifier: CarbonTableIdentifier,
thriftTableInfo: format.TableInfo,
carbonStorePath: String,
@@ -161,6 +183,30 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier).getPath
}
+ private def updateHiveMetaStoreForDataMap(newTableIdentifier: CarbonTableIdentifier,
+ oldTableIdentifier: CarbonTableIdentifier,
+ thriftTableInfo: format.TableInfo,
+ carbonStorePath: String,
+ sparkSession: SparkSession,
+ schemaConverter: ThriftWrapperSchemaConverterImpl) = {
+ val wrapperTableInfo = schemaConverter
+ .fromExternalToWrapperTableInfo(thriftTableInfo,
+ newTableIdentifier.getDatabaseName,
+ newTableIdentifier.getTableName,
+ carbonStorePath)
+ wrapperTableInfo.setStorePath(carbonStorePath)
+ val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier)
+ val schemaMetadataPath =
+ CarbonTablePath.getFolderContainingFile(carbonTablePath.getSchemaFilePath)
+ wrapperTableInfo.setMetaDataFilepath(schemaMetadataPath)
+ val dbName = oldTableIdentifier.getDatabaseName
+ val tableName = oldTableIdentifier.getTableName
+ sparkSession.catalog.refreshTable(TableIdentifier(tableName, Some(dbName)).quotedString)
+ removeTableFromMetadata(dbName, tableName)
+ CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo)
+ CarbonStorePath.getCarbonTablePath(carbonStorePath, newTableIdentifier).getPath
+ }
+
/**
* This method will is used to remove the evolution entry in case of failure.
*
@@ -168,7 +214,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
* @param thriftTableInfo
* @param sparkSession
*/
- override def revertTableSchema(carbonTableIdentifier: CarbonTableIdentifier,
+ override def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
thriftTableInfo: format.TableInfo,
tablePath: String)
(sparkSession: SparkSession): String = {
@@ -176,7 +222,23 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
val evolutionEntries = thriftTableInfo.fact_table.schema_evolution.schema_evolution_history
evolutionEntries.remove(evolutionEntries.size() - 1)
- updateHiveMetaStore(carbonTableIdentifier,
+ updateHiveMetaStoreForAlter(carbonTableIdentifier,
+ carbonTableIdentifier,
+ thriftTableInfo,
+ identifier.getStorePath,
+ sparkSession,
+ schemaConverter)
+ }
+
+ override def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier:
+ CarbonTableIdentifier,
+ thriftTableInfo: org.apache.carbondata.format.TableInfo,
+ tablePath: String)(sparkSession: SparkSession): String = {
+ val schemaConverter = new ThriftWrapperSchemaConverterImpl
+ val identifier = AbsoluteTableIdentifier.fromTablePath(tablePath)
+ val childSchemas = thriftTableInfo.dataMapSchemas
+ childSchemas.remove(childSchemas.size())
+ updateHiveMetaStoreForAlter(carbonTableIdentifier,
carbonTableIdentifier,
thriftTableInfo,
identifier.getStorePath,
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
index dcb43d1..24996ed 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
@@ -66,13 +66,26 @@ trait CarbonMetaStore {
* @param carbonStorePath
* @param sparkSession
*/
- def updateTableSchema(newTableIdentifier: CarbonTableIdentifier,
+ def updateTableSchemaForAlter(newTableIdentifier: CarbonTableIdentifier,
oldTableIdentifier: CarbonTableIdentifier,
thriftTableInfo: org.apache.carbondata.format.TableInfo,
schemaEvolutionEntry: SchemaEvolutionEntry,
carbonStorePath: String)(sparkSession: SparkSession): String
/**
+ * This method will overwrite the existing schema and update it with the given details
+ *
+ * @param newTableIdentifier
+ * @param thriftTableInfo
+ * @param carbonStorePath
+ * @param sparkSession
+ */
+ def updateTableSchemaForDataMap(newTableIdentifier: CarbonTableIdentifier,
+ oldTableIdentifier: CarbonTableIdentifier,
+ thriftTableInfo: org.apache.carbondata.format.TableInfo,
+ carbonStorePath: String)(sparkSession: SparkSession): String
+
+ /**
* This method will is used to remove the evolution entry in case of failure.
*
* @param carbonTableIdentifier
@@ -80,11 +93,15 @@ trait CarbonMetaStore {
* @param tablePath
* @param sparkSession
*/
- def revertTableSchema(carbonTableIdentifier: CarbonTableIdentifier,
+ def revertTableSchemaInAlterFailure(carbonTableIdentifier: CarbonTableIdentifier,
thriftTableInfo: org.apache.carbondata.format.TableInfo,
tablePath: String)
(sparkSession: SparkSession): String
+
+ def revertTableSchemaForPreAggCreationFailure(carbonTableIdentifier: CarbonTableIdentifier,
+ thriftTableInfo: org.apache.carbondata.format.TableInfo,
+ tablePath: String)(sparkSession: SparkSession): String
/**
* Prepare Thrift Schema from wrapper TableInfo and write to disk
*/
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/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 fc2ed41..bf21bc8 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
@@ -440,7 +440,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
// so checking the start of the string and taking the precision and scale.
// resetting the data type with decimal
if (f.dataType.getOrElse("").startsWith("decimal")) {
- val (precision, scale) = getScaleAndPrecision(col.dataType.catalogString)
+ val (precision, scale) = CommonUtil.getScaleAndPrecision(col.dataType.catalogString)
f.precision = precision
f.scale = scale
f.dataType = Some("decimal")
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/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 a53e71f..256477e 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
@@ -18,14 +18,15 @@ package org.apache.spark.sql.parser
import scala.collection.mutable
-import org.apache.spark.sql.{CarbonSession, SparkSession}
+import org.apache.spark.sql.{CarbonSession, DataFrame, Dataset, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, ParseException, SqlBaseParser}
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, CarbonCreateTableCommand, PartitionerField, TableModel}
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.preaaggregate.{CreatePreAggregateTableCommand, PreAggregateUtil}
import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
import org.apache.spark.sql.types.StructField
@@ -39,7 +40,7 @@ import org.apache.carbondata.spark.util.CommonUtil
*/
class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends AbstractSqlParser {
- val astBuilder = new CarbonSqlAstBuilder(conf)
+ val astBuilder = new CarbonSqlAstBuilder(conf, sparkSession: SparkSession)
private val substitutor = new VariableSubstitution(conf)
@@ -69,7 +70,8 @@ class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends Ab
}
}
-class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
+class CarbonSqlAstBuilder(conf: SQLConf, sparkSession: SparkSession)
+ extends SparkSqlAstBuilder(conf) {
val parser = new CarbonSpark2SqlParser
@@ -117,8 +119,18 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
val (partitionByStructFields, partitionFields) =
validateParitionFields(ctx, colNames, tableProperties)
- val fields = parser.getFields(colsStructFields ++ partitionByStructFields)
-
+ val isAggTable = tableProperties.get("parent").isDefined
+ var fields = parser.getFields(colsStructFields ++ partitionByStructFields)
+ val dfAndFieldRelationTuple = if (isAggTable) {
+ val selectQuery = Option(ctx.query).map(plan).get
+ val df = Dataset.ofRows(sparkSession, selectQuery)
+ val fieldRelationMap = PreAggregateUtil
+ .validateActualSelectPlanAndGetAttrubites(df.logicalPlan, source(ctx.query()))
+ fields = fieldRelationMap.keySet.toSeq
+ Some(df, fieldRelationMap)
+ } else {
+ None
+ }
// validate bucket fields
val bucketFields: Option[BucketFields] =
parser.getBucketFields(tableProperties, fields, options)
@@ -137,7 +149,14 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
isAlterFlow = false,
tableComment)
- CarbonCreateTableCommand(tableModel)
+ if(!isAggTable) {
+ CarbonCreateTableCommand(tableModel)
+ } else {
+ CreatePreAggregateTableCommand(tableModel,
+ dfAndFieldRelationTuple.get._1,
+ queryString = source(ctx.query).toString,
+ fieldRelationMap = dfAndFieldRelationTuple.get._2)
+ }
} else {
super.visitCreateTable(ctx)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/3d1d1ce8/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index 44f5a36..bda4eeb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -136,7 +136,7 @@ object AlterTableUtil {
val dbName = carbonTable.getDatabaseName
val tableName = carbonTable.getFactTableName
CarbonEnv.getInstance(sparkSession).carbonMetastore
- .updateTableSchema(carbonTable.getCarbonTableIdentifier,
+ .updateTableSchemaForAlter(carbonTable.getCarbonTableIdentifier,
carbonTable.getCarbonTableIdentifier,
thriftTable,
schemaEvolutionEntry,
@@ -211,7 +211,7 @@ object AlterTableUtil {
.renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
oldTableIdentifier.table)
val tableIdentifier = new CarbonTableIdentifier(database, oldTableIdentifier.table, tableId)
- metastore.revertTableSchema(tableIdentifier,
+ metastore.revertTableSchemaInAlterFailure(tableIdentifier,
tableInfo, carbonTablePath.getPath)(sparkSession)
metastore.removeTableFromMetadata(database, newTableName)
}
@@ -243,7 +243,7 @@ object AlterTableUtil {
val addedSchemas = evolutionEntryList.get(evolutionEntryList.size() - 1).added
thriftTable.fact_table.table_columns.removeAll(addedSchemas)
metastore
- .revertTableSchema(carbonTable.getCarbonTableIdentifier,
+ .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
}
}
@@ -278,7 +278,7 @@ object AlterTableUtil {
}
}
metastore
- .revertTableSchema(carbonTable.getCarbonTableIdentifier,
+ .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
}
}
@@ -316,7 +316,7 @@ object AlterTableUtil {
}
}
metastore
- .revertTableSchema(carbonTable.getCarbonTableIdentifier,
+ .revertTableSchemaInAlterFailure(carbonTable.getCarbonTableIdentifier,
thriftTable, carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
}
}
[7/9] carbondata git commit: [CARBONDATA-1576] Added create datamap
parser and saved to schema file
Posted by ra...@apache.org.
[CARBONDATA-1576] Added create datamap parser and saved to schema file
User can create datamap using the following syntax.
CREATE DATAMAP agg_sales
ON TABLE sales
USING org.apache.carbondata.datamap.AggregateDataMapHandler
DMPROPERTIES (
'KEY’=’VALUE’
) AS
SELECT order_time, count(user_id) FROM sales GROUP BY order_time
In the above syntax DMPROPERTIES and AS QUERY are optional.
This closes #1481
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/1c16afad
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/1c16afad
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/1c16afad
Branch: refs/heads/master
Commit: 1c16afad8decb42f28cb656e5f11664dcadf503b
Parents: 2b5faef
Author: ravipesala <ra...@gmail.com>
Authored: Sun Nov 12 17:08:09 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 14 00:48:16 2017 +0530
----------------------------------------------------------------------
.../ThriftWrapperSchemaConverterImpl.java | 44 +++---
.../metadata/schema/table/DataMapSchema.java | 14 +-
.../core/metadata/schema/table/TableInfo.java | 2 +-
.../core/metadata/schema/table/TableSchema.java | 12 +-
format/src/main/thrift/schema.thrift | 10 +-
.../preaggregate/TestPreAggCreateCommand.scala | 112 +++++++--------
.../preaggregate/TestPreAggregateDrop.scala | 28 ++--
.../preaggregate/TestPreAggregateLoad.scala | 19 +--
.../testsuite/datamap/TestDataMapCommand.scala | 95 +++++++++++++
.../iud/DeleteCarbonTableTestCase.scala | 8 +-
.../iud/UpdateCarbonTableTestCase.scala | 8 +-
.../spark/sql/catalyst/CarbonDDLSqlParser.scala | 5 +-
.../datamap/CarbonCreateDataMapCommand.scala | 68 +++++++++
.../CreatePreAggregateTableCommand.scala | 141 ++++++++-----------
.../preaaggregate/PreAggregateListeners.scala | 8 +-
.../preaaggregate/PreAggregateUtil.scala | 8 +-
.../spark/sql/hive/CarbonFileMetastore.scala | 5 +-
.../sql/parser/CarbonSpark2SqlParser.scala | 24 +++-
.../spark/sql/parser/CarbonSparkSqlParser.scala | 31 +---
.../AlterTableValidationTestCase.scala | 6 +-
.../vectorreader/ChangeDataTypeTestCases.scala | 8 +-
.../vectorreader/DropColumnTestCases.scala | 6 +-
.../apache/spark/util/CarbonCommandSuite.scala | 6 +-
23 files changed, 411 insertions(+), 257 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index 5ffc612..b914e06 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -322,17 +322,22 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
for (DataMapSchema wrapperChildSchema : wrapperChildSchemaList) {
org.apache.carbondata.format.DataMapSchema thriftChildSchema =
new org.apache.carbondata.format.DataMapSchema();
- org.apache.carbondata.format.RelationIdentifier relationIdentifier =
- new org.apache.carbondata.format.RelationIdentifier();
- relationIdentifier
- .setDatabaseName(wrapperChildSchema.getRelationIdentifier().getDatabaseName());
- relationIdentifier.setTableName(wrapperChildSchema.getRelationIdentifier().getTableName());
- relationIdentifier.setTableId(wrapperChildSchema.getRelationIdentifier().getTableId());
- thriftChildSchema.setRelationIdentifire(relationIdentifier);
+ if (wrapperChildSchema.getRelationIdentifier() != null) {
+ org.apache.carbondata.format.RelationIdentifier relationIdentifier =
+ new org.apache.carbondata.format.RelationIdentifier();
+ relationIdentifier.setDatabaseName(
+ wrapperChildSchema.getRelationIdentifier().getDatabaseName());
+ relationIdentifier.setTableName(wrapperChildSchema.getRelationIdentifier().getTableName());
+ relationIdentifier.setTableId(wrapperChildSchema.getRelationIdentifier().getTableId());
+ thriftChildSchema.setRelationIdentifire(relationIdentifier);
+ }
thriftChildSchema.setProperties(wrapperChildSchema.getProperties());
thriftChildSchema.setClassName(wrapperChildSchema.getClassName());
- thriftChildSchema.setChildTableSchema(
- fromWrapperToExternalTableSchema(wrapperChildSchema.getChildSchema()));
+ thriftChildSchema.setDataMapName(wrapperChildSchema.getDataMapName());
+ if (wrapperChildSchema.getChildSchema() != null) {
+ thriftChildSchema.setChildTableSchema(
+ fromWrapperToExternalTableSchema(wrapperChildSchema.getChildSchema()));
+ }
thriftChildSchemas.add(thriftChildSchema);
}
return thriftChildSchemas;
@@ -623,16 +628,19 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
@Override public DataMapSchema fromExternalToWrapperDataMapSchema(
org.apache.carbondata.format.DataMapSchema thriftDataMapSchema) {
- RelationIdentifier relationIdentifier =
- new RelationIdentifier(thriftDataMapSchema.getRelationIdentifire().getDatabaseName(),
- thriftDataMapSchema.getRelationIdentifire().getTableName(),
- thriftDataMapSchema.getRelationIdentifire().getTableId());
- DataMapSchema childSchema = new DataMapSchema(thriftDataMapSchema.getClassName());
+ DataMapSchema childSchema =
+ new DataMapSchema(thriftDataMapSchema.getDataMapName(), thriftDataMapSchema.getClassName());
childSchema.setProperties(thriftDataMapSchema.getProperties());
- childSchema.setChildSchema(
- fromExternalToWrapperTableSchema(thriftDataMapSchema.getChildTableSchema(),
- relationIdentifier.getTableName()));
- childSchema.setRelationIdentifier(relationIdentifier);
+ if (thriftDataMapSchema.getRelationIdentifire() != null) {
+ RelationIdentifier relationIdentifier =
+ new RelationIdentifier(thriftDataMapSchema.getRelationIdentifire().getDatabaseName(),
+ thriftDataMapSchema.getRelationIdentifire().getTableName(),
+ thriftDataMapSchema.getRelationIdentifire().getTableId());
+ childSchema.setRelationIdentifier(relationIdentifier);
+ childSchema.setChildSchema(
+ fromExternalToWrapperTableSchema(thriftDataMapSchema.getChildTableSchema(),
+ relationIdentifier.getTableName()));
+ }
return childSchema;
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
index 9c71e37..e0632d9 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java
@@ -30,6 +30,8 @@ public class DataMapSchema implements Serializable, Writable {
private static final long serialVersionUID = 6577149126264181553L;
+ private String dataMapName;
+
private String className;
private RelationIdentifier relationIdentifier;
@@ -43,7 +45,11 @@ public class DataMapSchema implements Serializable, Writable {
*/
private Map<String, String> properties;
- public DataMapSchema(String className) {
+ public DataMapSchema() {
+ }
+
+ public DataMapSchema(String dataMapName, String className) {
+ this.dataMapName = dataMapName;
this.className = className;
}
@@ -75,7 +81,12 @@ public class DataMapSchema implements Serializable, Writable {
this.properties = properties;
}
+ public String getDataMapName() {
+ return dataMapName;
+ }
+
@Override public void write(DataOutput out) throws IOException {
+ out.writeUTF(dataMapName);
out.writeUTF(className);
boolean isRelationIdentifierExists = null != relationIdentifier;
out.writeBoolean(isRelationIdentifierExists);
@@ -99,6 +110,7 @@ public class DataMapSchema implements Serializable, Writable {
}
@Override public void readFields(DataInput in) throws IOException {
+ this.dataMapName = in.readUTF();
this.className = in.readUTF();
boolean isRelationIdnentifierExists = in.readBoolean();
if (isRelationIdnentifierExists) {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index 44d8126..1d9e2ec 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -293,7 +293,7 @@ public class TableInfo implements Serializable, Writable {
if (isChildSchemaExists) {
short numberOfChildTable = in.readShort();
for (int i = 0; i < numberOfChildTable; i++) {
- DataMapSchema childSchema = new DataMapSchema(null);
+ DataMapSchema childSchema = new DataMapSchema();
childSchema.readFields(in);
dataMapSchemaList.add(childSchema);
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
index 5b013d1..714e0d8 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
@@ -255,21 +255,15 @@ public class TableSchema implements Serializable, Writable {
* Below method will be used to build child schema object which will be stored in
* parent table
*
- * @param className
- * @param databaseName
- * @param queryString
- * @param queryType
- *
- * @return datamap schema
*/
- public DataMapSchema buildChildSchema(String className, String databaseName, String queryString,
- String queryType) {
+ public DataMapSchema buildChildSchema(String dataMapName, String className, String databaseName,
+ String queryString, String queryType) {
RelationIdentifier relationIdentifier =
new RelationIdentifier(databaseName, tableName, tableId);
Map<String, String> properties = new HashMap<>();
properties.put("CHILD_SELECT QUERY", queryString);
properties.put("QUERYTYPE", queryType);
- DataMapSchema dataMapSchema = new DataMapSchema(className);
+ DataMapSchema dataMapSchema = new DataMapSchema(dataMapName, className);
dataMapSchema.setChildSchema(this);
dataMapSchema.setProperties(properties);
dataMapSchema.setRelationIdentifier(relationIdentifier);
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 4e9bb21..fc73cfb 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -187,15 +187,17 @@ struct ParentColumnTableRelation {
}
struct DataMapSchema {
+ // DataMap name
+ 1: required string dataMapName;
// class name
- 1: required string className;
+ 2: required string className;
// relation indentifier
- 2: optional RelationIdentifier relationIdentifire;
+ 3: optional RelationIdentifier relationIdentifire;
// in case of preaggregate it will be used to maintain the child schema
// which will be usefull in case of query and data load
- 3: optional TableSchema childTableSchema;
+ 4: optional TableSchema childTableSchema;
// to maintain properties like select query, query type like groupby, join
- 4: optional map<string, string> properties;
+ 5: optional map<string, string> properties;
}
struct TableInfo{
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
index 6120e88..ce2d1d2 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
@@ -16,100 +16,100 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
test("test pre agg create table One") {
- sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg1"), true, "preaggmain_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg1"), true, "preaggmain_b_sum")
- sql("drop table preagg1")
+ sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg1"), true, "preaggmain_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg1"), true, "preaggmain_b_sum")
+ sql("drop table PreAggMain_preagg1")
}
test("test pre agg create table Two") {
- sql("create table preagg2 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a as a1,sum(b) from PreAggMain group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg2"), true, "preaggmain_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg2"), true, "preaggmain_b_sum")
- sql("drop table preagg2")
+ sql("create datamap preagg2 on table PreAggMain using 'preaggregate' as select a as a1,sum(b) from PreAggMain group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), true, "preaggmain_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg2"), true, "preaggmain_b_sum")
+ sql("drop table PreAggMain_preagg2")
}
test("test pre agg create table Three") {
- sql("create table preagg3 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) as sum from PreAggMain group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg3"), true, "preaggmain_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg3"), true, "preaggmain_b_sum")
- sql("drop table preagg3")
+ sql("create datamap preagg3 on table PreAggMain using 'preaggregate' as select a,sum(b) as sum from PreAggMain group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg3"), true, "preaggmain_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg3"), true, "preaggmain_b_sum")
+ sql("drop table PreAggMain_preagg3")
}
test("test pre agg create table four") {
- sql("create table preagg4 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a as a1,sum(b) as sum from PreAggMain group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg4"), true, "preaggmain_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg4"), true, "preaggmain_b_sum")
- sql("drop table preagg4")
+ sql("create datamap preagg4 on table PreAggMain using 'preaggregate' as select a as a1,sum(b) as sum from PreAggMain group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg4"), true, "preaggmain_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain_preagg4"), true, "preaggmain_b_sum")
+ sql("drop table PreAggMain_preagg4")
}
test("test pre agg create table five") {
- sql("create table preagg11 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a,sum(b) from PreAggMain1 group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "preaggmain1_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "preaggmain1_b_sum")
- checkExistence(sql("DESCRIBE FORMATTED preagg11"), true, "DICTIONARY")
- sql("drop table preagg11")
+ sql("create datamap preagg11 on table PreAggMain1 using 'preaggregate'as select a,sum(b) from PreAggMain1 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "preaggmain1_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "preaggmain1_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg11"), true, "DICTIONARY")
+ sql("drop table PreAggMain1_preagg11")
}
test("test pre agg create table six") {
- sql("create table preagg12 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a as a1,sum(b) from PreAggMain1 group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "preaggmain1_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "preaggmain1_b_sum")
- checkExistence(sql("DESCRIBE FORMATTED preagg12"), true, "DICTIONARY")
- sql("drop table preagg12")
+ sql("create datamap preagg12 on table PreAggMain1 using 'preaggregate' as select a as a1,sum(b) from PreAggMain1 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "preaggmain1_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "preaggmain1_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg12"), true, "DICTIONARY")
+ sql("drop table PreAggMain1_preagg12")
}
test("test pre agg create table seven") {
- sql("create table preagg13 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a,sum(b) as sum from PreAggMain1 group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "preaggmain1_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "preaggmain1_b_sum")
- checkExistence(sql("DESCRIBE FORMATTED preagg13"), true, "DICTIONARY")
- sql("drop table preagg13")
+ sql("create datamap preagg13 on table PreAggMain1 using 'preaggregate' as select a,sum(b) as sum from PreAggMain1 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg13"), true, "preaggmain1_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg13"), true, "preaggmain1_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg13"), true, "DICTIONARY")
+ sql("drop table PreAggMain1_preagg13")
}
test("test pre agg create table eight") {
- sql("create table preagg14 stored BY 'carbondata' tblproperties('parent'='PreAggMain1') as select a as a1,sum(b) as sum from PreAggMain1 group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "preaggmain1_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "preaggmain1_b_sum")
- checkExistence(sql("DESCRIBE FORMATTED preagg14"), true, "DICTIONARY")
- sql("drop table preagg14")
+ sql("create datamap preagg14 on table PreAggMain1 using 'preaggregate' as select a as a1,sum(b) as sum from PreAggMain1 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "preaggmain1_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "preaggmain1_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain1_preagg14"), true, "DICTIONARY")
+ sql("drop table PreAggMain1_preagg14")
}
test("test pre agg create table nine") {
- sql("create table preagg15 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a,avg(b) from PreAggMain2 group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_b_sum")
- checkExistence(sql("DESCRIBE FORMATTED preagg15"), true, "preaggmain2_b_count")
- sql("drop table preagg15")
+ sql("create datamap preagg15 on table PreAggMain2 using 'preaggregate' as select a,avg(b) from PreAggMain2 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg15"), true, "preaggmain2_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg15"), true, "preaggmain2_b_sum")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg15"), true, "preaggmain2_b_count")
+ sql("drop table PreAggMain2_preagg15")
}
test("test pre agg create table ten") {
- sql("create table preagg16 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,max(b) from PreAggMain2 group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg16"), true, "preaggmain2_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg16"), true, "preaggmain2_b_max")
- sql("drop table preagg16")
+ sql("create datamap preagg16 on table PreAggMain2 using 'preaggregate' as select a as a1,max(b) from PreAggMain2 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg16"), true, "preaggmain2_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg16"), true, "preaggmain2_b_max")
+ sql("drop table PreAggMain2_preagg16")
}
test("test pre agg create table eleven") {
- sql("create table preagg17 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a,min(b) from PreAggMain2 group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg17"), true, "preaggmain2_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg17"), true, "preaggmain2_b_min")
- sql("drop table preagg17")
+ sql("create datamap preagg17 on table PreAggMain2 using 'preaggregate' as select a,min(b) from PreAggMain2 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg17"), true, "preaggmain2_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg17"), true, "preaggmain2_b_min")
+ sql("drop table PreAggMain2_preagg17")
}
test("test pre agg create table twelve") {
- sql("create table preagg18 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,count(b) from PreAggMain2 group by a")
- checkExistence(sql("DESCRIBE FORMATTED preagg18"), true, "preaggmain2_a")
- checkExistence(sql("DESCRIBE FORMATTED preagg18"), true, "preaggmain2_b_count")
- sql("drop table preagg18")
+ sql("create datamap preagg18 on table PreAggMain2 using 'preaggregate' as select a as a1,count(b) from PreAggMain2 group by a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg18"), true, "preaggmain2_a")
+ checkExistence(sql("DESCRIBE FORMATTED PreAggMain2_preagg18"), true, "preaggmain2_b_count")
+ sql("drop table PreAggMain2_preagg18")
}
test("test pre agg create table thirteen") {
try {
sql(
- "create table preagg19 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,count(distinct b) from PreAggMain2 group by a")
+ "create datamap preagg19 on table PreAggMain2 using 'preaggregate' as select a as a1,count(distinct b) from PreAggMain2 group by a")
assert(false)
} catch {
case _: Exception =>
@@ -120,7 +120,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
test("test pre agg create table fourteen") {
try {
sql(
- "create table preagg20 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,sum(distinct b) from PreAggMain2 group by a")
+ "create datamap preagg20 on table PreAggMain2 using 'preaggregate' as select a as a1,sum(distinct b) from PreAggMain2 group by a")
assert(false)
} catch {
case _: Exception =>
@@ -131,7 +131,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
test("test pre agg create table fifteen") {
try {
sql(
- "create table preagg21 stored BY 'carbondata' tblproperties('parent'='PreAggMain2') as select a as a1,sum(b) from PreAggMain2 where a='vishal' group by a")
+ "create datamap preagg21 on table PreAggMain2 using 'preaggregate' as select a as a1,sum(b) from PreAggMain2 where a='vishal' group by a")
assert(false)
} catch {
case _: Exception =>
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
index 4dad3e1..fee73a7 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
@@ -24,44 +24,44 @@ class TestPreAggregateDrop extends QueryTest with BeforeAndAfterAll {
override def beforeAll {
sql("drop table if exists maintable")
- sql("drop table if exists preagg1")
- sql("drop table if exists preagg2")
+ sql("drop table if exists maintable_preagg1")
+ sql("drop table if exists maintable_preagg2")
sql("create table maintable (a string, b string, c string) stored by 'carbondata'")
}
test("create and drop preaggregate table") {
sql(
- "create table preagg1 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+ "create datamap preagg1 on table maintable using 'preaggregate' as select" +
" a,sum(b) from maintable group by a")
- sql("drop table if exists preagg1")
- checkExistence(sql("show tables"), false, "preagg1")
+ sql("drop table if exists maintable_preagg1")
+ checkExistence(sql("show tables"), false, "maintable_preagg1")
}
test("dropping 1 aggregate table should not drop others") {
sql(
- "create table preagg1 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+ "create datamap preagg1 on table maintable using 'preaggregate' as select" +
" a,sum(b) from maintable group by a")
sql(
- "create table preagg2 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+ "create datamap preagg2 on table maintable using 'preaggregate' as select" +
" a,sum(c) from maintable group by a")
- sql("drop table if exists preagg2")
+ sql("drop table if exists maintable_preagg2")
val showTables = sql("show tables")
- checkExistence(showTables, false, "preagg2")
- checkExistence(showTables, true, "preagg1")
+ checkExistence(showTables, false, "maintable_preagg2")
+ checkExistence(showTables, true, "maintable_preagg1")
}
test("drop main table and check if preaggreagte is deleted") {
sql(
- "create table preagg2 stored BY 'carbondata' tblproperties('parent'='maintable') as select" +
+ "create datamap preagg2 on table maintable using 'preaggregate' as select" +
" a,sum(c) from maintable group by a")
sql("drop table if exists maintable")
- checkExistence(sql("show tables"), false, "preagg1", "maintable", "preagg2")
+ checkExistence(sql("show tables"), false, "maintable_preagg1", "maintable", "maintable_preagg2")
}
override def afterAll() {
sql("drop table if exists maintable")
- sql("drop table if exists preagg1")
- sql("drop table if exists preagg2")
+ sql("drop table if exists maintable_preagg1")
+ sql("drop table if exists maintable_preagg2")
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
index 1f576c5..5ac3534 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateLoad.scala
@@ -23,33 +23,28 @@ import org.scalatest.{BeforeAndAfterAll, Ignore}
@Ignore
class TestPreAggregateLoad extends QueryTest with BeforeAndAfterAll {
-
+
val testData = s"$resourcesPath/sample.csv"
-
+
override def beforeAll(): Unit = {
sql("DROP TABLE IF EXISTS maintable")
}
private def createAllAggregateTables(parentTableName: String): Unit = {
sql(
- s"""create table ${ parentTableName }_preagg_sum stored BY 'carbondata' tblproperties
- |('parent'='$parentTableName') as select id,sum(age) from $parentTableName group by id"""
+ s"""create datamap preagg_sum on table $parentTableName using 'preaggregate' as select id,sum(age) from $parentTableName group by id"""
.stripMargin)
sql(
- s"""create table ${ parentTableName }_preagg_avg stored BY 'carbondata' tblproperties
- |('parent'='$parentTableName') as select id,avg(age) from $parentTableName group by id"""
+ s"""create datamap preagg_avg on table $parentTableName using 'preaggregate' as select id,avg(age) from $parentTableName group by id"""
.stripMargin)
sql(
- s"""create table ${ parentTableName }_preagg_count stored BY 'carbondata' tblproperties
- |('parent'='$parentTableName') as select id,count(age) from $parentTableName group by id"""
+ s"""create datamap preagg_count on table $parentTableName using 'preaggregate' as select id,count(age) from $parentTableName group by id"""
.stripMargin)
sql(
- s"""create table ${ parentTableName }_preagg_min stored BY 'carbondata' tblproperties
- |('parent'='$parentTableName') as select id,min(age) from $parentTableName group by id"""
+ s"""create datamap preagg_min on table $parentTableName using 'preaggregate' as select id,min(age) from $parentTableName group by id"""
.stripMargin)
sql(
- s"""create table ${ parentTableName }_preagg_max stored BY 'carbondata' tblproperties
- |('parent'='$parentTableName') as select id,max(age) from $parentTableName group by id"""
+ s"""create datamap preagg_max on table $parentTableName using 'preaggregate' as select id,max(age) from $parentTableName group by id"""
.stripMargin)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
new file mode 100644
index 0000000..2512def
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.datamap
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.metadata.CarbonMetadata
+
+class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
+
+ override def beforeAll {
+ sql("drop table if exists datamaptest")
+ sql("create table datamaptest (a string, b string, c string) stored by 'carbondata'")
+ }
+
+
+ test("test datamap create") {
+ sql("create datamap datamap1 on table datamaptest using 'new.class'")
+ val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+ assert(table != null)
+ val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+ assert(dataMapSchemaList.size() == 1)
+ assert(dataMapSchemaList.get(0).getDataMapName.equals("datamap1"))
+ assert(dataMapSchemaList.get(0).getClassName.equals("new.class"))
+ }
+
+ test("test datamap create with dmproperties") {
+ sql("create datamap datamap2 on table datamaptest using 'new.class' dmproperties('key'='value')")
+ val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+ assert(table != null)
+ val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+ assert(dataMapSchemaList.size() == 2)
+ assert(dataMapSchemaList.get(1).getDataMapName.equals("datamap2"))
+ assert(dataMapSchemaList.get(1).getClassName.equals("new.class"))
+ assert(dataMapSchemaList.get(1).getProperties.get("key").equals("value"))
+ }
+
+ test("test datamap create with existing name") {
+ intercept[Exception] {
+ sql(
+ "create datamap datamap2 on table datamaptest using 'new.class' dmproperties('key'='value')")
+ }
+ val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+ assert(table != null)
+ val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+ assert(dataMapSchemaList.size() == 2)
+ }
+
+ test("test datamap create with preagg") {
+ sql("drop table if exists datamap3")
+ sql(
+ "create datamap datamap3 on table datamaptest using 'preaggregate' dmproperties('key'='value') as select count(a) from datamaptest")
+ val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+ assert(table != null)
+ val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+ assert(dataMapSchemaList.size() == 3)
+ assert(dataMapSchemaList.get(2).getDataMapName.equals("datamap3"))
+ assert(dataMapSchemaList.get(2).getProperties.get("key").equals("value"))
+ assert(dataMapSchemaList.get(2).getChildSchema.getTableName.equals("datamaptest_datamap3"))
+ }
+
+ test("test datamap create with preagg with duplicate name") {
+ intercept[Exception] {
+ sql("drop table if exists datamap2")
+ sql(
+ "create datamap datamap2 on table datamaptest using 'preaggregate' dmproperties('key'='value') as select count(a) from datamaptest")
+
+ }
+ val table = CarbonMetadata.getInstance().getCarbonTable("default_datamaptest")
+ assert(table != null)
+ val dataMapSchemaList = table.getTableInfo.getDataMapSchemaList
+ assert(dataMapSchemaList.size() == 3)
+ }
+
+
+ override def afterAll {
+ sql("drop table if exists datamaptest")
+ }
+}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
index a2bd6aa..f7e93af 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
@@ -131,17 +131,17 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
test("test if delete is unsupported for pre-aggregate tables") {
sql("drop table if exists preaggMain")
- sql("drop table if exists preagg1")
+ sql("drop table if exists preaggmain_preagg1")
sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
- sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+ sql("create datamap preagg1 on table PreAggMain USING 'preaggregate' as select a,sum(b) from PreAggMain group by a")
intercept[RuntimeException] {
sql("delete from preaggmain where a = 'abc'").show()
}.getMessage.contains("Delete operation is not supported for tables")
intercept[RuntimeException] {
- sql("delete from preagg1 where preaggmain_a = 'abc'").show()
+ sql("delete from preaggmain_preagg1 where preaggmain_a = 'abc'").show()
}.getMessage.contains("Delete operation is not supported for pre-aggregate table")
sql("drop table if exists preaggMain")
- sql("drop table if exists preagg1")
+ sql("drop table if exists preaggmain_preagg1")
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index 4c43ec0..fd5f144 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -515,17 +515,17 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
test("test if update is unsupported for pre-aggregate tables") {
sql("drop table if exists preaggMain")
- sql("drop table if exists preagg1")
+ sql("drop table if exists preaggMain_preagg1")
sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
- sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+ sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
intercept[RuntimeException] {
sql("update preaggmain set (a)=('a')").show
}.getMessage.contains("Update operation is not supported for tables")
intercept[RuntimeException] {
- sql("update preagg1 set (a)=('a')").show
+ sql("update preaggMain_preagg1 set (a)=('a')").show
}.getMessage.contains("Update operation is not supported for pre-aggregate table")
sql("drop table if exists preaggMain")
- sql("drop table if exists preagg1")
+ sql("drop table if exists preaggMain_preagg1")
}
override def afterAll {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index bb80bce..42447da 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -170,6 +170,9 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
protected val CHANGE = carbonKeyWord("CHANGE")
protected val TBLPROPERTIES = carbonKeyWord("TBLPROPERTIES")
protected val ID = carbonKeyWord("ID")
+ protected val DATAMAP = carbonKeyWord("DATAMAP")
+ protected val ON = carbonKeyWord("ON")
+ protected val DMPROPERTIES = carbonKeyWord("DMPROPERTIES")
protected val doubleQuotedString = "\"([^\"]+)\"".r
protected val singleQuotedString = "'([^']+)'".r
@@ -750,7 +753,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
* @param dbName
* @return Option of String
*/
- protected def convertDbNameToLowerCase(dbName: Option[String]): Option[String] = {
+ def convertDbNameToLowerCase(dbName: Option[String]): Option[String] = {
dbName match {
case Some(databaseName) => Some(convertDbNameToLowerCase(databaseName))
case None => dbName
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
new file mode 100644
index 0000000..8b6e3b9
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.datamap
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.preaaggregate.{CreatePreAggregateTableCommand, PreAggregateUtil}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
+
+/**
+ * Below command class will be used to create datamap on table
+ * and updating the parent table about the datamap information
+ *
+ * @param queryString
+ */
+case class CarbonCreateDataMapCommand(
+ dataMapName: String,
+ tableIdentifier: TableIdentifier,
+ dmClassName: String,
+ dmproperties: Map[String, String],
+ queryString: Option[String])
+ extends RunnableCommand with SchemaProcessCommand {
+
+ override def run(sparkSession: SparkSession): Seq[Row] = {
+ processSchema(sparkSession)
+ }
+
+ override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+ val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+ if (dmClassName.equals("org.apache.carbondata.datamap.AggregateDataMapHandler") ||
+ dmClassName.equalsIgnoreCase("preaggregate")) {
+ CreatePreAggregateTableCommand(dataMapName,
+ tableIdentifier,
+ dmClassName,
+ dmproperties,
+ queryString.get).run(sparkSession)
+ } else {
+ val dataMapSchema = new DataMapSchema(dataMapName, dmClassName)
+ dataMapSchema.setProperties(new java.util.HashMap[String, String](dmproperties.asJava))
+ val dbName = GetDB.getDatabaseName(tableIdentifier.database, sparkSession)
+ // upadting the parent table about dataschema
+ PreAggregateUtil.updateMainTable(dbName, tableIdentifier.table, dataMapSchema, sparkSession)
+ }
+ LOGGER.audit(s"DataMap ${dataMapName} successfully added to Table ${tableIdentifier.table}")
+ Seq.empty
+ }
+}
+
+
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index e12cbb9..ebf6273 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -16,16 +16,14 @@
*/
package org.apache.spark.sql.execution.command.preaaggregate
+import scala.collection.mutable
+import scala.collection.JavaConverters._
+
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.execution.command._
-
-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
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
/**
* Below command class will be used to create pre-aggregate table
@@ -34,17 +32,14 @@ import org.apache.carbondata.core.util.CarbonUtil
* 1. failed to create pre aggregate table.
* 2. failed to update main table
*
- * @param cm
- * @param dataFrame
- * @param createDSTable
* @param queryString
*/
case class CreatePreAggregateTableCommand(
- cm: TableModel,
- dataFrame: DataFrame,
- createDSTable: Boolean = true,
- queryString: String,
- fieldRelationMap: scala.collection.mutable.LinkedHashMap[Field, DataMapField])
+ dataMapName: String,
+ parentTableIdentifier: TableIdentifier,
+ dmClassName: String,
+ dmproperties: Map[String, String],
+ queryString: String)
extends RunnableCommand with SchemaProcessCommand {
override def run(sparkSession: SparkSession): Seq[Row] = {
@@ -52,87 +47,61 @@ case class CreatePreAggregateTableCommand(
}
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 df = sparkSession.sql(queryString)
+ val fieldRelationMap = PreAggregateUtil
+ .validateActualSelectPlanAndGetAttrubites(df.logicalPlan, queryString)
+ val fields = fieldRelationMap.keySet.toSeq
+ val tableProperties = mutable.Map[String, String]()
+ dmproperties.foreach(t => tableProperties.put(t._1, t._2))
+ // Create the aggregation table name with parent table name prefix
+ val tableIdentifier = TableIdentifier(
+ parentTableIdentifier.table +"_" + dataMapName, parentTableIdentifier.database)
+ // prepare table model of the collected tokens
+ val tableModel: TableModel = new CarbonSpark2SqlParser().prepareTableModel(false,
+ new CarbonSpark2SqlParser().convertDbNameToLowerCase(tableIdentifier.database),
+ tableIdentifier.table.toLowerCase,
+ fields,
+ Seq(),
+ tableProperties,
+ None,
+ false,
+ None)
+
// getting the parent table
- val parentTable = PreAggregateUtil.getParentCarbonTable(dataFrame.logicalPlan)
+ val parentTable = PreAggregateUtil.getParentCarbonTable(df.logicalPlan)
// getting the table name
val parentTableName = parentTable.getFactTableName
// getting the db name of parent table
val parentDbName = parentTable.getDatabaseName
+
+ assert(parentTableName.equalsIgnoreCase(parentTableIdentifier.table))
// updating the relation identifier, this will be stored in child table
// which can be used during dropping of pre-aggreate table as parent table will
// also get updated
- cm.parentTable = Some(parentTable)
- cm.dataMapRelation = Some(fieldRelationMap)
- 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) """)
- // child schema object which will be updated on parent table about the
- val childSchema = tableInfo.getFactTable
- .buildChildSchema("", tableInfo.getDatabaseName, queryString, "AGGREGATION")
- // upadting the parent table about child table
- PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)
- val loadAvailable = PreAggregateUtil
- .checkMainTableLoad(parentTable)
- if (loadAvailable) {
- sparkSession.sql(s"insert into ${ cm.databaseName }.${ cm.tableName } $queryString")
- }
- } 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
- }
+ tableModel.parentTable = Some(parentTable)
+ tableModel.dataMapRelation = Some(fieldRelationMap)
+ CarbonCreateTableCommand(tableModel).run(sparkSession)
+ try {
+ val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore.
+ lookupRelation(tableIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
+ val tableInfo = relation.tableMeta.carbonTable.getTableInfo
+ // child schema object which will be updated on parent table about the
+ val childSchema = tableInfo.getFactTable.buildChildSchema(
+ dataMapName, "", tableInfo.getDatabaseName, queryString, "AGGREGATION")
+ dmproperties.foreach(f => childSchema.getProperties.put(f._1, f._2))
+ // updating the parent table about child table
+ PreAggregateUtil.updateMainTable(parentDbName, parentTableName, childSchema, sparkSession)
+ val loadAvailable = PreAggregateUtil.checkMainTableLoad(parentTable)
+ if (loadAvailable) {
+ sparkSession.sql(
+ s"insert into ${ tableModel.databaseName }.${ tableModel.tableName } $queryString")
}
+ } catch {
+ case e: Exception =>
+ sparkSession.sql(
+ s"""DROP TABLE IF EXISTS ${ tableModel.databaseName }.${ tableModel.tableName }""")
+ 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/1c16afad/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
index 2ce97fe..8271e57 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -41,9 +41,11 @@ object DropPreAggregateTablePostListener extends OperationEventListener {
!carbonTable.get.getTableInfo.getDataMapSchemaList.isEmpty) {
val childSchemas = carbonTable.get.getTableInfo.getDataMapSchemaList
for (childSchema: DataMapSchema <- childSchemas.asScala) {
- CarbonDropTableCommand(ifExistsSet = true,
- Some(childSchema.getRelationIdentifier.getDatabaseName),
- childSchema.getRelationIdentifier.getTableName).run(sparkSession)
+ if (childSchema.getRelationIdentifier != null) {
+ CarbonDropTableCommand(ifExistsSet = true,
+ Some(childSchema.getRelationIdentifier.getDatabaseName),
+ childSchema.getRelationIdentifier.getTableName).run(sparkSession)
+ }
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index b35b525..b926705 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.execution.command.preaaggregate
import scala.collection.mutable.ListBuffer
+import scala.collection.JavaConverters._
import org.apache.spark.SparkConf
import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
@@ -303,6 +304,10 @@ object PreAggregateUtil {
tableName,
carbonTable.getStorePath)
numberOfCurrentChild = wrapperTableInfo.getDataMapSchemaList.size
+ if (wrapperTableInfo.getDataMapSchemaList.asScala.
+ exists(f => f.getDataMapName.equalsIgnoreCase(childSchema.getDataMapName))) {
+ throw new Exception("Duplicate datamap")
+ }
wrapperTableInfo.getDataMapSchemaList.add(childSchema)
val thriftTable = schemaConverter
.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
@@ -446,7 +451,6 @@ object PreAggregateUtil {
}
def checkMainTableLoad(carbonTable: CarbonTable): Boolean = {
- SegmentStatusManager.readLoadMetadata(
- carbonTable.getMetaDataFilepath).nonEmpty
+ SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath).nonEmpty
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index ac75fa7..a5e1ec4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -465,8 +465,9 @@ class CarbonFileMetastore extends CarbonMetaStore {
}
val childSchemaIterator = childSchemas.iterator()
while (childSchemaIterator.hasNext) {
- val childSchema = childSchemaIterator.next()
- if (childSchema.getChildSchema.equals(childCarbonTable.getTableInfo.getFactTable)) {
+ val childSchema = childSchemaIterator.next().getChildSchema
+ if (childSchema != null &&
+ childSchema.equals(childCarbonTable.getTableInfo.getFactTable)) {
childSchemaIterator.remove()
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 bf21bc8..46a2515 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,12 @@ import scala.collection.mutable
import scala.language.implicitConversions
import org.apache.spark.sql.{DeleteRecords, ShowLoadsCommand, UpdateTable}
-import org.apache.spark.sql.catalyst.CarbonDDLSqlParser
+import org.apache.spark.sql.catalyst.{CarbonDDLSqlParser, TableIdentifier}
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.datamap.CarbonCreateDataMapCommand
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.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand, CarbonAlterTableDropColumnCommand}
@@ -66,7 +67,8 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
protected lazy val start: Parser[LogicalPlan] = explainPlan | startCommand
protected lazy val startCommand: Parser[LogicalPlan] =
- loadManagement|showLoads|alterTable|restructure|updateTable|deleteRecords|alterPartition
+ loadManagement|showLoads|alterTable|restructure|updateTable|deleteRecords|
+ alterPartition|datamapManagement
protected lazy val loadManagement: Parser[LogicalPlan] =
deleteLoadsByID | deleteLoadsByLoadDate | cleanFiles | loadDataNew
@@ -77,6 +79,9 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
protected lazy val alterPartition: Parser[LogicalPlan] =
alterAddPartition | alterSplitPartition | alterDropPartition
+ protected lazy val datamapManagement: Parser[LogicalPlan] =
+ createDataMap
+
protected lazy val alterAddPartition: Parser[LogicalPlan] =
ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (ADD ~> PARTITION ~>
"(" ~> repsep(stringLit, ",") <~ ")") <~ opt(";") ^^ {
@@ -121,6 +126,21 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
AlterTableCompactionCommand(altertablemodel)
}
+ /**
+ * The syntax of datamap creation is as follows.
+ * CREATE DATAMAP datamapName ON TABLE tableName USING 'DataMapClassName'
+ * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
+ */
+ protected lazy val createDataMap: Parser[LogicalPlan] =
+ CREATE ~> DATAMAP ~> ident ~ (ON ~ TABLE) ~ (ident <~ ".").? ~ ident ~
+ (USING ~> stringLit) ~ (DMPROPERTIES ~> "(" ~> repsep(loadOptions, ",") <~ ")").? ~
+ (AS ~> restInput).? <~ opt(";") ^^ {
+ case dmname ~ ontable ~ dbName ~ tableName ~ className ~ dmprops ~ query =>
+ val map = dmprops.getOrElse(List[(String, String)]()).toMap[String, String]
+ CarbonCreateDataMapCommand(dmname,
+ TableIdentifier(tableName, dbName), className, map, query)
+ }
+
protected lazy val deleteRecords: Parser[LogicalPlan] =
(DELETE ~> FROM ~> table) ~ restInput.? <~ opt(";") ^^ {
case table ~ rest =>
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/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 256477e..5c51156 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
@@ -25,8 +25,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._
-import org.apache.spark.sql.execution.command.preaaggregate.{CreatePreAggregateTableCommand, PreAggregateUtil}
+import org.apache.spark.sql.execution.command.{BucketFields, CarbonCreateTableCommand, PartitionerField, TableModel}
import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
import org.apache.spark.sql.types.StructField
@@ -40,7 +39,7 @@ import org.apache.carbondata.spark.util.CommonUtil
*/
class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends AbstractSqlParser {
- val astBuilder = new CarbonSqlAstBuilder(conf, sparkSession: SparkSession)
+ val astBuilder = new CarbonSqlAstBuilder(conf)
private val substitutor = new VariableSubstitution(conf)
@@ -70,8 +69,7 @@ class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends Ab
}
}
-class CarbonSqlAstBuilder(conf: SQLConf, sparkSession: SparkSession)
- extends SparkSqlAstBuilder(conf) {
+class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
val parser = new CarbonSpark2SqlParser
@@ -119,18 +117,8 @@ class CarbonSqlAstBuilder(conf: SQLConf, sparkSession: SparkSession)
val (partitionByStructFields, partitionFields) =
validateParitionFields(ctx, colNames, tableProperties)
- val isAggTable = tableProperties.get("parent").isDefined
- var fields = parser.getFields(colsStructFields ++ partitionByStructFields)
- val dfAndFieldRelationTuple = if (isAggTable) {
- val selectQuery = Option(ctx.query).map(plan).get
- val df = Dataset.ofRows(sparkSession, selectQuery)
- val fieldRelationMap = PreAggregateUtil
- .validateActualSelectPlanAndGetAttrubites(df.logicalPlan, source(ctx.query()))
- fields = fieldRelationMap.keySet.toSeq
- Some(df, fieldRelationMap)
- } else {
- None
- }
+ val fields = parser.getFields(colsStructFields ++ partitionByStructFields)
+
// validate bucket fields
val bucketFields: Option[BucketFields] =
parser.getBucketFields(tableProperties, fields, options)
@@ -149,14 +137,7 @@ class CarbonSqlAstBuilder(conf: SQLConf, sparkSession: SparkSession)
isAlterFlow = false,
tableComment)
- if(!isAggTable) {
- CarbonCreateTableCommand(tableModel)
- } else {
- CreatePreAggregateTableCommand(tableModel,
- dfAndFieldRelationTuple.get._1,
- queryString = source(ctx.query).toString,
- fieldRelationMap = dfAndFieldRelationTuple.get._2)
- }
+ CarbonCreateTableCommand(tableModel)
} else {
super.visitCreateTable(ctx)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
index 0274605..b798379 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
@@ -493,13 +493,13 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
test("test to check if new parent table name is reflected in pre-aggregate tables") {
sql("drop table if exists preaggMain")
sql("drop table if exists preaggmain_new")
- sql("drop table if exists preagg1")
+ sql("drop table if exists preaggMain_preagg1")
sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
sql(
- "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+ "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
" a,sum(b) from PreAggMain group by a")
intercept[RuntimeException] {
- sql("alter table preagg1 rename to preagg2")
+ sql("alter table PreAggMain_preagg1 rename to preagg2")
}.getMessage.contains("Rename operation for pre-aggregate table is not supported.")
intercept[RuntimeException] {
sql("alter table preaggmain rename to preaggmain_new")
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
index 67ea21e..31a8b6c 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/ChangeDataTypeTestCases.scala
@@ -149,19 +149,19 @@ class ChangeDataTypeTestCases extends Spark2QueryTest with BeforeAndAfterAll {
test("test data type change for with pre-aggregate table should throw exception") {
sql("drop table if exists preaggMain")
- sql("drop table if exists preagg1")
+ sql("drop table if exists PreAggMain_preagg1")
sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
sql(
- "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+ "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
" a,sum(b) from PreAggMain group by a")
intercept[RuntimeException] {
sql("alter table preaggmain drop columns(a)").show
}.getMessage.contains("exists in pre-aggregate table")
intercept[RuntimeException] {
- sql("alter table preagg1 drop columns(a)").show
+ sql("alter table PreAggMain_preagg1 drop columns(a)").show
}.getMessage.contains("cannot be dropped")
sql("drop table if exists preaggMain")
- sql("drop table if exists preagg1")
+ sql("drop table if exists PreAggMain_preagg1")
}
override def afterAll {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
index 1a1d5d5..b887771 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/DropColumnTestCases.scala
@@ -100,10 +100,10 @@ class DropColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
test("test dropping of column in pre-aggregate should throw exception") {
sql("drop table if exists preaggMain")
- sql("drop table if exists preagg1")
+ sql("drop table if exists preaggMain_preagg1")
sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
sql(
- "create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select" +
+ "create datamap preagg1 on table PreAggMain using 'preaggregate' as select" +
" a,sum(b) from PreAggMain group by a")
sql("alter table preaggmain drop columns(c)").show
checkExistence(sql("desc table preaggmain"), false, "c")
@@ -111,7 +111,7 @@ class DropColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
sql("alter table preaggmain drop columns(a)").show
}.getMessage.contains("cannot be dropped")
sql("drop table if exists preaggMain")
- sql("drop table if exists preagg1")
+ sql("drop table if exists preaggMain_preagg1")
}
override def afterAll {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/1c16afad/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
index c65bcc4..f79ea2d 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
@@ -143,14 +143,14 @@ class CarbonCommandSuite extends Spark2QueryTest with BeforeAndAfterAll {
test("test if delete segments by id is unsupported for pre-aggregate tables") {
dropTable("preaggMain")
- dropTable("preagg1")
+ dropTable("preaggMain_preagg1")
sql("create table preaggMain (a string, b string, c string) stored by 'carbondata'")
- sql("create table preagg1 stored BY 'carbondata' tblproperties('parent'='PreAggMain') as select a,sum(b) from PreAggMain group by a")
+ sql("create datamap preagg1 on table PreAggMain using 'preaggregate' as select a,sum(b) from PreAggMain group by a")
intercept[UnsupportedOperationException] {
sql("delete from table preaggMain where segment.id in (1,2)")
}.getMessage.contains("Delete segment operation is not supported on tables")
intercept[UnsupportedOperationException] {
- sql("delete from table preagg1 where segment.id in (1,2)")
+ sql("delete from table preaggMain_preagg1 where segment.id in (1,2)")
}.getMessage.contains("Delete segment operation is not supported on pre-aggregate tables")
dropTable("preaggMain")
dropTable("preagg1")
[8/9] carbondata git commit: [CARBONDATA-1520] [PreAgg] Support
pre-aggregate table load
Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
index 5dbd177..22d15d9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
@@ -25,11 +25,15 @@ import org.apache.carbondata.core.cache.dictionary.Dictionary;
import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
import org.apache.carbondata.core.dictionary.client.DictionaryClient;
import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnIdentifier;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
import org.apache.carbondata.processing.datatypes.ArrayDataType;
import org.apache.carbondata.processing.datatypes.GenericDataType;
import org.apache.carbondata.processing.datatypes.PrimitiveDataType;
@@ -76,12 +80,40 @@ public class FieldEncoderFactory {
isEmptyBadRecord);
} else if (dataField.getColumn().hasEncoding(Encoding.DICTIONARY) &&
!dataField.getColumn().isComplex()) {
- return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier, nullFormat,
- index, client, useOnePass, storePath, localCache, isEmptyBadRecord);
+ DictionaryColumnUniqueIdentifier identifier = null;
+ // if parent column table relation is not null then it's a child table
+ // in case of child table it will use parent table dictionary
+ if (null == dataField.getColumn().getColumnSchema().getParentColumnTableRelations()
+ || dataField.getColumn().getColumnSchema().getParentColumnTableRelations().isEmpty()) {
+ identifier = new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+ dataField.getColumn().getColumnIdentifier(), dataField.getColumn().getDataType(),
+ CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
+ return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier,
+ nullFormat, index, client, useOnePass, storePath, localCache, isEmptyBadRecord,
+ identifier);
+ } else {
+ ParentColumnTableRelation parentColumnTableRelation =
+ dataField.getColumn().getColumnSchema().getParentColumnTableRelations().get(0);
+ RelationIdentifier relationIdentifier =
+ parentColumnTableRelation
+ .getRelationIdentifier();
+ CarbonTableIdentifier parentTableIdentifier =
+ new CarbonTableIdentifier(relationIdentifier.getDatabaseName(),
+ relationIdentifier.getTableName(), relationIdentifier.getTableId());
+ ColumnIdentifier parentColumnIdentifier =
+ new ColumnIdentifier(parentColumnTableRelation.getColumnId(), null,
+ dataField.getColumn().getDataType());
+ identifier =
+ new DictionaryColumnUniqueIdentifier(parentTableIdentifier, parentColumnIdentifier,
+ dataField.getColumn().getDataType(),
+ CarbonStorePath.getCarbonTablePath(storePath, parentTableIdentifier));
+ return new DictionaryFieldConverterImpl(dataField, cache, parentTableIdentifier,
+ nullFormat, index, null, false, storePath, null, isEmptyBadRecord, identifier);
+ }
} else if (dataField.getColumn().isComplex()) {
return new ComplexFieldConverterImpl(
createComplexType(dataField, cache, carbonTableIdentifier,
- client, useOnePass, storePath, localCache), index);
+ client, useOnePass, storePath, localCache), index);
} else {
return new NonDictionaryFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord);
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/cc0e6f1e/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index 6a156a6..acd7fed 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -163,6 +163,16 @@ public class CarbonLoadModel implements Serializable {
*/
private String globalSortPartitions;
+ private boolean isAggLoadRequest;
+
+ public boolean isAggLoadRequest() {
+ return isAggLoadRequest;
+ }
+
+ public void setAggLoadRequest(boolean aggLoadRequest) {
+ isAggLoadRequest = aggLoadRequest;
+ }
+
/**
* get escape char
*
@@ -353,6 +363,7 @@ public class CarbonLoadModel implements Serializable {
copy.sortScope = sortScope;
copy.batchSortSizeInMb = batchSortSizeInMb;
copy.badRecordsLocation = badRecordsLocation;
+ copy.isAggLoadRequest = isAggLoadRequest;
return copy;
}
@@ -399,6 +410,7 @@ public class CarbonLoadModel implements Serializable {
copy.isEmptyDataBadRecord = isEmptyDataBadRecord;
copy.sortScope = sortScope;
copy.batchSortSizeInMb = batchSortSizeInMb;
+ copy.isAggLoadRequest = isAggLoadRequest;
return copy;
}
@@ -449,6 +461,7 @@ public class CarbonLoadModel implements Serializable {
copyObj.sortScope = sortScope;
copyObj.batchSortSizeInMb = batchSortSizeInMb;
copyObj.badRecordsLocation = badRecordsLocation;
+ copyObj.isAggLoadRequest = isAggLoadRequest;
return copyObj;
}