akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378948275 ########## File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java ########## @@ -290,4 +301,71 @@ public boolean isTimeSeries() { public void setTimeSeries(boolean timeSeries) { isTimeSeries = timeSeries; } + + public boolean supportIncrementalBuild() { + String prop = getProperties().get(DataMapProperty.FULL_REFRESH); + return prop == null || prop.equalsIgnoreCase("false"); + } + + public String getPropertiesAsString() { + String[] properties = getProperties().entrySet().stream() + // ignore internal used property + .filter(p -> + !p.getKey().equalsIgnoreCase(DataMapProperty.DEFERRED_REBUILD) && + !p.getKey().equalsIgnoreCase(DataMapProperty.CHILD_SELECT_QUERY) && + !p.getKey().equalsIgnoreCase(DataMapProperty.QUERY_TYPE) && + !p.getKey().equalsIgnoreCase(DataMapProperty.FULL_REFRESH)) + .map(p -> "'" + p.getKey() + "'='" + p.getValue() + "'") + .sorted() + .toArray(String[]::new); + return Strings.mkString(properties, ","); + } + + public String getTable() { + return relationIdentifier.getDatabaseName() + "." + relationIdentifier.getTableName(); Review comment: use point constants from CarbonCommonConstants ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378948860 ########## File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java ########## @@ -290,4 +301,71 @@ public boolean isTimeSeries() { public void setTimeSeries(boolean timeSeries) { isTimeSeries = timeSeries; } + + public boolean supportIncrementalBuild() { + String prop = getProperties().get(DataMapProperty.FULL_REFRESH); + return prop == null || prop.equalsIgnoreCase("false"); + } + + public String getPropertiesAsString() { + String[] properties = getProperties().entrySet().stream() + // ignore internal used property + .filter(p -> + !p.getKey().equalsIgnoreCase(DataMapProperty.DEFERRED_REBUILD) && + !p.getKey().equalsIgnoreCase(DataMapProperty.CHILD_SELECT_QUERY) && + !p.getKey().equalsIgnoreCase(DataMapProperty.QUERY_TYPE) && + !p.getKey().equalsIgnoreCase(DataMapProperty.FULL_REFRESH)) + .map(p -> "'" + p.getKey() + "'='" + p.getValue() + "'") + .sorted() + .toArray(String[]::new); + return Strings.mkString(properties, ","); + } + + public String getTable() { Review comment: i think can be renamed to `getUniqueTableName` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378960831 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVDataMapProvider.scala ########## @@ -207,3 +207,7 @@ class MVDataMapProvider( override def supportRebuild(): Boolean = true } + +object MVDataMapProvider { Review comment: No need to create this object right, we already have `DataMapClassProvider` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378956132 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVAnalyzerRule.scala ########## @@ -56,7 +56,7 @@ class MVAnalyzerRule(sparkSession: SparkSession) extends Rule[LogicalPlan] { // first check if any mv UDF is applied it is present is in plan // then call is from create MV so no need to transform the query plan // TODO Add different UDF name - case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase(CarbonEnv.MV_SKIP_RULE_UDF) => + case al@Alias(udf: ScalaUDF, name) if name.equalsIgnoreCase(MVUdf.MV_SKIP_RULE_UDF) => Review comment: can we rename to `MvUDF` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378962286 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVExtension.scala ########## @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.carbondata.mv.extension + +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, SQLConf} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule + +import org.apache.carbondata.mv.rewrite.MVUdf +import org.apache.carbondata.mv.timeseries.TimeSeriesFunction + +// Materialized View extension for Apache Spark +// +// Following SQL command are added: +// 1. CREATE MATERIALIZED VIEW +// 2. DROP MATERIALIZED VIEW +// 3. SHOW MATERIALIZED VIEW +// 4. REFRESH MATERIALIZED VIEW +// +// Following optimizer rules are added: +// 1. Rewrite SQL statement by matching existing MV and +// select the lowest cost MV +// +class MVExtension extends (SparkSessionExtensions => Unit) { + + override def apply(extensions: SparkSessionExtensions): Unit = { + // MV parser + extensions.injectParser( + (sparkSession: SparkSession, parser: ParserInterface) => + new MVExtensionSqlParser(new SQLConf, sparkSession, parser)) + + // MV optimizer rules + extensions.injectPostHocResolutionRule( + (session: SparkSession) => OptimizerRule(session) ) + } +} + +case class OptimizerRule(session: SparkSession) extends Rule[LogicalPlan] { Review comment: can we rename it as `MVOptimizerRule` to be more specific ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378961450 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVExtension.scala ########## @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.carbondata.mv.extension + +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, SQLConf} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule + +import org.apache.carbondata.mv.rewrite.MVUdf +import org.apache.carbondata.mv.timeseries.TimeSeriesFunction + +// Materialized View extension for Apache Spark Review comment: please use Java Doc comment style. like /** * * **/ ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378965526 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVExtension.scala ########## @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.carbondata.mv.extension + +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions, SQLConf} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule + +import org.apache.carbondata.mv.rewrite.MVUdf +import org.apache.carbondata.mv.timeseries.TimeSeriesFunction + +// Materialized View extension for Apache Spark +// +// Following SQL command are added: +// 1. CREATE MATERIALIZED VIEW +// 2. DROP MATERIALIZED VIEW +// 3. SHOW MATERIALIZED VIEW +// 4. REFRESH MATERIALIZED VIEW +// +// Following optimizer rules are added: +// 1. Rewrite SQL statement by matching existing MV and +// select the lowest cost MV +// +class MVExtension extends (SparkSessionExtensions => Unit) { + + override def apply(extensions: SparkSessionExtensions): Unit = { + // MV parser + extensions.injectParser( + (sparkSession: SparkSession, parser: ParserInterface) => + new MVExtensionSqlParser(new SQLConf, sparkSession, parser)) + + // MV optimizer rules + extensions.injectPostHocResolutionRule( + (session: SparkSession) => OptimizerRule(session) ) + } +} + +case class OptimizerRule(session: SparkSession) extends Rule[LogicalPlan] { + self => + + var initialized = false + + override def apply(plan: LogicalPlan): LogicalPlan = { + if (!initialized) { + self.synchronized { + if (!initialized) { + initialized = true + + addMVUdf(session) + + val sessionState = session.sessionState + val field = sessionState.getClass.getDeclaredField("optimizer") + field.setAccessible(true) + field.set(sessionState, + new MVRules(session, sessionState.catalog, sessionState.optimizer)) + } + } + } + plan + } + + private def addMVUdf(sparkSession: SparkSession) = { + // added for handling MV table creation. when user will fire create ddl for Review comment: now we do not have preaggregate right, can we modifiy this comment or better to remove if not required ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378973665 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/command/CreateMaterializedViewCommand.scala ########## @@ -0,0 +1,106 @@ +/* + * 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.mv.extension.command + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.command._ + +import org.apache.carbondata.common.exceptions.sql.MalformedMaterializedViewException +import org.apache.carbondata.common.logging.LogServiceFactory +import org.apache.carbondata.core.datamap.{DataMapProvider, DataMapStoreManager} +import org.apache.carbondata.core.datamap.status.DataMapStatusManager +import org.apache.carbondata.core.metadata.schema.datamap.DataMapProperty +import org.apache.carbondata.core.metadata.schema.table.DataMapSchema +import org.apache.carbondata.core.util.CarbonProperties +import org.apache.carbondata.datamap.DataMapManager +import org.apache.carbondata.events._ +import org.apache.carbondata.mv.extension.MVDataMapProvider + +/** + * Create Materialized View Command implementation + * It will create the MV table, load the MV table (if deferred rebuild is false), + * and register the MV schema in [[DataMapStoreManager]] + */ +case class CreateMaterializedViewCommand( + mvName: String, + properties: Map[String, String], + queryString: Option[String], + ifNotExistsSet: Boolean = false, + deferredRebuild: Boolean = false) + extends AtomicRunnableCommand { + + private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) + private var dataMapProvider: DataMapProvider = _ + private var dataMapSchema: DataMapSchema = _ + + override def processMetadata(sparkSession: SparkSession): Seq[Row] = { + + setAuditInfo(Map("mvName" -> mvName) ++ properties) + + dataMapSchema = new DataMapSchema(mvName, MVDataMapProvider.MV_PROVIDER_NAME) + val property = properties.map(x => (x._1.trim, x._2.trim)).asJava + val javaMap = new java.util.HashMap[String, String](property) + javaMap.put(DataMapProperty.DEFERRED_REBUILD, deferredRebuild.toString) + dataMapSchema.setProperties(javaMap) + + dataMapProvider = DataMapManager.get.getDataMapProvider(null, dataMapSchema, sparkSession) + if (DataMapStoreManager.getInstance().getAllDataMapSchemas.asScala + .exists(_.getDataMapName.equalsIgnoreCase(dataMapSchema.getDataMapName))) { + if (!ifNotExistsSet) { + throw new MalformedMaterializedViewException( Review comment: can use s" <string> " instead of `+` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378963805 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/MatchMaker.scala ########## @@ -17,7 +17,7 @@ package org.apache.carbondata.mv.rewrite -import org.apache.spark.Logging +import org.apache.spark.internal.Logging Review comment: revert the change if not required ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378968380 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVHelper.scala ########## @@ -0,0 +1,487 @@ +/* + * 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.mv.extension + +import java.util + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.sql.{CarbonEnv, SparkSession} +import org.apache.spark.sql.catalyst.{CarbonParserUtil, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, Coalesce, Expression, Literal, ScalaUDF} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average} +import org.apache.spark.sql.catalyst.plans.logical.{Join, Limit, LogicalPlan} +import org.apache.spark.sql.execution.command.{Field, PartitionerField, TableModel, TableNewProcessor} +import org.apache.spark.sql.execution.command.table.{CarbonCreateTableCommand, CarbonDropTableCommand} +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.types.{ArrayType, DateType, MapType, StructType} +import org.apache.spark.util.{DataMapUtil, PartitionUtils} + +import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException +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.metadata.datatype.DataTypes +import org.apache.carbondata.core.metadata.schema.datamap.{DataMapClassProvider, DataMapProperty} +import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema, RelationIdentifier} +import org.apache.carbondata.core.statusmanager.SegmentStatusManager +import org.apache.carbondata.datamap.DataMapManager +import org.apache.carbondata.mv.plans.modular.{GroupBy, ModularPlan} +import org.apache.carbondata.mv.plans.util.SQLBuilder +import org.apache.carbondata.mv.rewrite.{SummaryDatasetCatalog, Utils} +import org.apache.carbondata.mv.timeseries.{TimeSeriesFunction, TimeSeriesUtil} +import org.apache.carbondata.spark.util.CommonUtil + +/** + * Utility for MV datamap operations. + */ +object MVHelper { + + def createMVDataMap( + sparkSession: SparkSession, + dataMapSchema: DataMapSchema, + queryString: String, + ifNotExistsSet: Boolean = false): Unit = { + val dmProperties = dataMapSchema.getProperties.asScala + if (dmProperties.contains("streaming") && dmProperties("streaming").equalsIgnoreCase("true")) { + throw new MalformedCarbonCommandException( + s"Materialized view does not support streaming" + ) + } + val mvUtil = new MVUtil + mvUtil.validateDMProperty(dmProperties) + val logicalPlan = Utils.dropDummyFunc( + MVParser.getMVPlan(queryString, sparkSession)) + // if there is limit in MV ctas query string, throw exception, as its not a valid usecase + logicalPlan match { + case Limit(_, _) => + throw new MalformedCarbonCommandException("Materialized view does not support the query " + + "with limit") + case _ => + } + val selectTables = getTables(logicalPlan) + if (selectTables.isEmpty) { + throw new MalformedCarbonCommandException( + s"Non-Carbon table does not support creating MV datamap") + } + val modularPlan = validateMVQuery(sparkSession, logicalPlan) + val updatedQueryWithDb = modularPlan.asCompactSQL + val (timeSeriesColumn, granularity): (String, String) = validateMVTimeSeriesQuery( + logicalPlan, + dataMapSchema) + val fullRebuild = isFullReload(logicalPlan) + var counter = 0 + // the ctas query can have duplicate columns, so we should take distinct and create fields, + // so that it won't fail during create mv table + val fields = logicalPlan.output.map { attr => + if (attr.dataType.isInstanceOf[ArrayType] || attr.dataType.isInstanceOf[StructType] || + attr.dataType.isInstanceOf[MapType]) { + throw new UnsupportedOperationException( + s"MV datamap is not supported for complex datatype columns and complex datatype return " + + s"types of function :" + attr.name) + } + val name = updateColumnName(attr, counter) + counter += 1 + val rawSchema = '`' + name + '`' + ' ' + attr.dataType.typeName + if (attr.dataType.typeName.startsWith("decimal")) { + val (precision, scale) = CommonUtil.getScaleAndPrecision(attr.dataType.catalogString) + Field(column = name, + dataType = Some(attr.dataType.typeName), + name = Some(name), + children = None, + precision = precision, + scale = scale, + rawSchema = rawSchema) + } else { + Field(column = name, + dataType = Some(attr.dataType.typeName), + name = Some(name), + children = None, + rawSchema = rawSchema) + } + }.distinct + + val tableProperties = mutable.Map[String, String]() + val parentTables = new util.ArrayList[String]() + val parentTablesList = new util.ArrayList[CarbonTable](selectTables.size) + selectTables.foreach { selectTable => + val mainCarbonTable = try { + Some(CarbonEnv.getCarbonTable(selectTable.identifier.database, + selectTable.identifier.table)(sparkSession)) + } catch { + // Exception handling if it's not a CarbonTable + case ex: Exception => + throw new MalformedCarbonCommandException( + s"Non-Carbon table does not support creating MV datamap") + } + if (!mainCarbonTable.get.getTableInfo.isTransactionalTable) { + throw new MalformedCarbonCommandException("Unsupported operation on NonTransactional table") + } + if (mainCarbonTable.get.isChildTableForMV) { + throw new MalformedCarbonCommandException( + "Cannot create Datamap on child table " + mainCarbonTable.get.getTableUniqueName) + } + parentTables.add(mainCarbonTable.get.getTableName) + if (!mainCarbonTable.isEmpty && mainCarbonTable.get.isStreamingSink) { + throw new MalformedCarbonCommandException( + s"Streaming table does not support creating materialized view") + } + parentTablesList.add(mainCarbonTable.get) + } + + // Check if load is in progress in any of the parent table mapped to the datamap + parentTablesList.asScala.foreach { + parentTable => + if (SegmentStatusManager.isLoadInProgressInTable(parentTable)) { + throw new UnsupportedOperationException( + "Cannot create mv datamap table when insert is in progress on parent table: " + + parentTable.getTableName) + } + } + + tableProperties.put(CarbonCommonConstants.DATAMAP_NAME, dataMapSchema.getDataMapName) + tableProperties.put(CarbonCommonConstants.PARENT_TABLES, parentTables.asScala.mkString(",")) + + val finalModularPlan = new SQLBuilder(modularPlan).SQLizer.execute(modularPlan) + val fieldRelationMap = mvUtil.getFieldsAndDataMapFieldsFromPlan(finalModularPlan, + getLogicalRelation(logicalPlan)) + // If dataMap is mapped to single main table, then inherit table properties from main table, + // else, will use default table properties. If DMProperties contains table properties, then + // table properties of datamap table will be updated + if (parentTablesList.size() == 1) { + DataMapUtil + .inheritTablePropertiesFromMainTable( + parentTablesList.get(0), + fields, + fieldRelationMap, + tableProperties) + if (granularity != null) { + val timeSeriesDataType = parentTablesList + .get(0) + .getTableInfo + .getFactTable + .getListOfColumns + .asScala + .filter(columnSchema => columnSchema.getColumnName + .equalsIgnoreCase(timeSeriesColumn)) + .head + .getDataType + if (timeSeriesDataType.equals(DataTypes.DATE) || + timeSeriesDataType.equals(DataTypes.TIMESTAMP)) { + // if data type is of Date type, then check if given granularity is valid for date type + if (timeSeriesDataType.equals(DataTypes.DATE)) { + TimeSeriesUtil.validateTimeSeriesGranularityForDate(granularity) + } + } else { + throw new MalformedCarbonCommandException( + "TimeSeries Column must be of TimeStamp or Date type") + } + } + } + dmProperties.foreach(t => tableProperties.put(t._1, t._2)) + val usePartitioning = dmProperties.getOrElse("partitioning", "true").toBoolean + var partitionerFields: Seq[PartitionerField] = Seq.empty + // Inherit partition from parent table if datamap is mapped to single parent table + if (parentTablesList.size() == 1) { + val partitionInfo = parentTablesList.get(0).getPartitionInfo + val parentPartitionColumns = if (!usePartitioning) { + Seq.empty + } else if (parentTablesList.get(0).isHivePartitionTable) { + partitionInfo.getColumnSchemaList.asScala.map(_.getColumnName) + } else { + Seq() + } + partitionerFields = PartitionUtils + .getPartitionerFields(parentPartitionColumns, fieldRelationMap) + } + + var order = 0 + val columnOrderMap = new java.util.HashMap[Integer, String]() + if (partitionerFields.nonEmpty) { + fields.foreach { field => + columnOrderMap.put(order, field.column) + order += 1 + } + } + + // TODO Use a proper DB + val tableIdentifier = + TableIdentifier(dataMapSchema.getDataMapName + "_table", + selectTables.head.identifier.database) + // prepare table model of the collected tokens + val tableModel: TableModel = CarbonParserUtil.prepareTableModel( + ifNotExistPresent = ifNotExistsSet, + CarbonParserUtil.convertDbNameToLowerCase(tableIdentifier.database), + tableIdentifier.table.toLowerCase, + fields, + partitionerFields, + tableProperties, + None, + isAlterFlow = false, + None) + + val tablePath = if (dmProperties.contains("path")) { + dmProperties("path") + } else { + CarbonEnv.getTablePath(tableModel.databaseNameOp, tableModel.tableName)(sparkSession) + } + CarbonCreateTableCommand(TableNewProcessor(tableModel), + tableModel.ifNotExistsSet, Some(tablePath), isVisible = false).run(sparkSession) + + // Map list of main table columns mapped to datamap table and add to dataMapSchema + val mainTableToColumnsMap = new java.util.HashMap[String, util.Set[String]]() + val mainTableFieldIterator = fieldRelationMap.values.asJava.iterator() + while (mainTableFieldIterator.hasNext) { + val value = mainTableFieldIterator.next() + value.columnTableRelationList.foreach { + columnTableRelation => + columnTableRelation.foreach { + mainTable => + if (null == mainTableToColumnsMap.get(mainTable.parentTableName)) { + val columns = new util.HashSet[String]() + columns.add(mainTable.parentColumnName.toLowerCase()) + mainTableToColumnsMap.put(mainTable.parentTableName, columns) + } else { + mainTableToColumnsMap.get(mainTable.parentTableName) + .add(mainTable.parentColumnName.toLowerCase()) + } + } + } + } + dataMapSchema.setMainTableColumnList(mainTableToColumnsMap) + dataMapSchema.setColumnsOrderMap(columnOrderMap) + if (null != granularity && null != timeSeriesColumn) { + dataMapSchema.setCtasQuery(queryString) + dataMapSchema.setTimeSeries(true) + } else { + dataMapSchema.setCtasQuery(updatedQueryWithDb) + } + dataMapSchema + .setRelationIdentifier(new RelationIdentifier(tableIdentifier.database.get, + tableIdentifier.table, + CarbonEnv.getCarbonTable(tableIdentifier)(sparkSession).getTableId)) + + val parentIdents = selectTables.map { table => + val relationIdentifier = new RelationIdentifier(table.database, table.identifier.table, "") + relationIdentifier.setTablePath(FileFactory.getUpdatedFilePath(table.location.toString)) + relationIdentifier + } + dataMapSchema.getRelationIdentifier.setTablePath(tablePath) + dataMapSchema.setParentTables(new util.ArrayList[RelationIdentifier](parentIdents.asJava)) + dataMapSchema.getProperties.put(DataMapProperty.FULL_REFRESH, fullRebuild.toString) + try { + DataMapStoreManager.getInstance().saveDataMapSchema(dataMapSchema) + } catch { + case ex: Exception => + val dropTableCommand = CarbonDropTableCommand(true, + new Some[String](dataMapSchema.getRelationIdentifier.getDatabaseName), + dataMapSchema.getRelationIdentifier.getTableName, + true) + dropTableCommand.run(sparkSession) + throw ex + } + } + + private def validateMVQuery( + sparkSession: SparkSession, + logicalPlan: LogicalPlan): ModularPlan = { + val dataMapProvider = DataMapManager.get().getDataMapProvider(null, + new DataMapSchema("", DataMapClassProvider.MV.getShortName), sparkSession) + var catalog = DataMapStoreManager.getInstance().getDataMapCatalog(dataMapProvider, + DataMapClassProvider.MV.getShortName).asInstanceOf[SummaryDatasetCatalog] + if (catalog == null) { + catalog = new SummaryDatasetCatalog(sparkSession) + } + val modularPlan = + catalog.mvSession.sessionState.modularizer.modularize( + catalog.mvSession.sessionState.optimizer.execute(logicalPlan)).next().semiHarmonized + + // Only queries which can be select , predicate , join, group by and having queries. + if (!modularPlan.isSPJGH) { + throw new UnsupportedOperationException("MV is not supported for this query") + } + val isValid = modularPlan match { + case g: GroupBy => + // Make sure all predicates are present in projections. + g.predicateList.forall{p => + g.outputList.exists{ + case a: Alias => + a.semanticEquals(p) || a.child.semanticEquals(p) + case other => other.semanticEquals(p) + } + } + case _ => true + } + if (!isValid) { + throw new UnsupportedOperationException( + "Group by columns must be present in project columns") + } + if (catalog.isMVWithSameQueryPresent(logicalPlan)) { + throw new UnsupportedOperationException("MV with same query present") + } + + var expressionValid = true + modularPlan.transformExpressions { + case coal@Coalesce(_) if coal.children.exists( + exp => exp.isInstanceOf[AggregateExpression]) => + expressionValid = false + coal + } + + if (!expressionValid) { + throw new UnsupportedOperationException("MV doesn't support Coalesce") + } + modularPlan + } + + def getUpdatedName(name: String, counter: Int): String = { + var updatedName = name.replace("(", "_") + .replace(")", "") Review comment: i think better to replace with Constants defined in CarbonCommonConstants and define the static constants for others ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378973214 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/command/CreateMaterializedViewCommand.scala ########## @@ -0,0 +1,106 @@ +/* + * 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.mv.extension.command + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.command._ + +import org.apache.carbondata.common.exceptions.sql.MalformedMaterializedViewException +import org.apache.carbondata.common.logging.LogServiceFactory +import org.apache.carbondata.core.datamap.{DataMapProvider, DataMapStoreManager} +import org.apache.carbondata.core.datamap.status.DataMapStatusManager +import org.apache.carbondata.core.metadata.schema.datamap.DataMapProperty +import org.apache.carbondata.core.metadata.schema.table.DataMapSchema +import org.apache.carbondata.core.util.CarbonProperties +import org.apache.carbondata.datamap.DataMapManager +import org.apache.carbondata.events._ +import org.apache.carbondata.mv.extension.MVDataMapProvider + +/** + * Create Materialized View Command implementation + * It will create the MV table, load the MV table (if deferred rebuild is false), + * and register the MV schema in [[DataMapStoreManager]] + */ +case class CreateMaterializedViewCommand( + mvName: String, + properties: Map[String, String], + queryString: Option[String], + ifNotExistsSet: Boolean = false, + deferredRebuild: Boolean = false) + extends AtomicRunnableCommand { + + private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) + private var dataMapProvider: DataMapProvider = _ + private var dataMapSchema: DataMapSchema = _ + + override def processMetadata(sparkSession: SparkSession): Seq[Row] = { + + setAuditInfo(Map("mvName" -> mvName) ++ properties) + + dataMapSchema = new DataMapSchema(mvName, MVDataMapProvider.MV_PROVIDER_NAME) + val property = properties.map(x => (x._1.trim, x._2.trim)).asJava + val javaMap = new java.util.HashMap[String, String](property) Review comment: no need to create on more map,`property` is already a java map, can put in same and set to datamapSchema ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378968659 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVHelper.scala ########## @@ -0,0 +1,487 @@ +/* + * 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.mv.extension + +import java.util + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.sql.{CarbonEnv, SparkSession} +import org.apache.spark.sql.catalyst.{CarbonParserUtil, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Cast, Coalesce, Expression, Literal, ScalaUDF} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Average} +import org.apache.spark.sql.catalyst.plans.logical.{Join, Limit, LogicalPlan} +import org.apache.spark.sql.execution.command.{Field, PartitionerField, TableModel, TableNewProcessor} +import org.apache.spark.sql.execution.command.table.{CarbonCreateTableCommand, CarbonDropTableCommand} +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.types.{ArrayType, DateType, MapType, StructType} +import org.apache.spark.util.{DataMapUtil, PartitionUtils} + +import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException +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.metadata.datatype.DataTypes +import org.apache.carbondata.core.metadata.schema.datamap.{DataMapClassProvider, DataMapProperty} +import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema, RelationIdentifier} +import org.apache.carbondata.core.statusmanager.SegmentStatusManager +import org.apache.carbondata.datamap.DataMapManager +import org.apache.carbondata.mv.plans.modular.{GroupBy, ModularPlan} +import org.apache.carbondata.mv.plans.util.SQLBuilder +import org.apache.carbondata.mv.rewrite.{SummaryDatasetCatalog, Utils} +import org.apache.carbondata.mv.timeseries.{TimeSeriesFunction, TimeSeriesUtil} +import org.apache.carbondata.spark.util.CommonUtil + +/** + * Utility for MV datamap operations. + */ +object MVHelper { + + def createMVDataMap( + sparkSession: SparkSession, + dataMapSchema: DataMapSchema, + queryString: String, + ifNotExistsSet: Boolean = false): Unit = { + val dmProperties = dataMapSchema.getProperties.asScala + if (dmProperties.contains("streaming") && dmProperties("streaming").equalsIgnoreCase("true")) { + throw new MalformedCarbonCommandException( + s"Materialized view does not support streaming" + ) + } + val mvUtil = new MVUtil + mvUtil.validateDMProperty(dmProperties) + val logicalPlan = Utils.dropDummyFunc( + MVParser.getMVPlan(queryString, sparkSession)) + // if there is limit in MV ctas query string, throw exception, as its not a valid usecase + logicalPlan match { + case Limit(_, _) => + throw new MalformedCarbonCommandException("Materialized view does not support the query " + + "with limit") + case _ => + } + val selectTables = getTables(logicalPlan) + if (selectTables.isEmpty) { + throw new MalformedCarbonCommandException( + s"Non-Carbon table does not support creating MV datamap") + } + val modularPlan = validateMVQuery(sparkSession, logicalPlan) + val updatedQueryWithDb = modularPlan.asCompactSQL + val (timeSeriesColumn, granularity): (String, String) = validateMVTimeSeriesQuery( + logicalPlan, + dataMapSchema) + val fullRebuild = isFullReload(logicalPlan) + var counter = 0 + // the ctas query can have duplicate columns, so we should take distinct and create fields, + // so that it won't fail during create mv table + val fields = logicalPlan.output.map { attr => + if (attr.dataType.isInstanceOf[ArrayType] || attr.dataType.isInstanceOf[StructType] || + attr.dataType.isInstanceOf[MapType]) { + throw new UnsupportedOperationException( + s"MV datamap is not supported for complex datatype columns and complex datatype return " + + s"types of function :" + attr.name) + } + val name = updateColumnName(attr, counter) + counter += 1 + val rawSchema = '`' + name + '`' + ' ' + attr.dataType.typeName + if (attr.dataType.typeName.startsWith("decimal")) { + val (precision, scale) = CommonUtil.getScaleAndPrecision(attr.dataType.catalogString) + Field(column = name, + dataType = Some(attr.dataType.typeName), + name = Some(name), + children = None, + precision = precision, + scale = scale, + rawSchema = rawSchema) + } else { + Field(column = name, + dataType = Some(attr.dataType.typeName), + name = Some(name), + children = None, + rawSchema = rawSchema) + } + }.distinct + + val tableProperties = mutable.Map[String, String]() + val parentTables = new util.ArrayList[String]() + val parentTablesList = new util.ArrayList[CarbonTable](selectTables.size) + selectTables.foreach { selectTable => + val mainCarbonTable = try { + Some(CarbonEnv.getCarbonTable(selectTable.identifier.database, + selectTable.identifier.table)(sparkSession)) + } catch { + // Exception handling if it's not a CarbonTable + case ex: Exception => + throw new MalformedCarbonCommandException( + s"Non-Carbon table does not support creating MV datamap") + } + if (!mainCarbonTable.get.getTableInfo.isTransactionalTable) { + throw new MalformedCarbonCommandException("Unsupported operation on NonTransactional table") + } + if (mainCarbonTable.get.isChildTableForMV) { + throw new MalformedCarbonCommandException( + "Cannot create Datamap on child table " + mainCarbonTable.get.getTableUniqueName) + } + parentTables.add(mainCarbonTable.get.getTableName) + if (!mainCarbonTable.isEmpty && mainCarbonTable.get.isStreamingSink) { + throw new MalformedCarbonCommandException( + s"Streaming table does not support creating materialized view") + } + parentTablesList.add(mainCarbonTable.get) + } + + // Check if load is in progress in any of the parent table mapped to the datamap + parentTablesList.asScala.foreach { + parentTable => + if (SegmentStatusManager.isLoadInProgressInTable(parentTable)) { + throw new UnsupportedOperationException( + "Cannot create mv datamap table when insert is in progress on parent table: " + + parentTable.getTableName) + } + } + + tableProperties.put(CarbonCommonConstants.DATAMAP_NAME, dataMapSchema.getDataMapName) + tableProperties.put(CarbonCommonConstants.PARENT_TABLES, parentTables.asScala.mkString(",")) + + val finalModularPlan = new SQLBuilder(modularPlan).SQLizer.execute(modularPlan) + val fieldRelationMap = mvUtil.getFieldsAndDataMapFieldsFromPlan(finalModularPlan, + getLogicalRelation(logicalPlan)) + // If dataMap is mapped to single main table, then inherit table properties from main table, + // else, will use default table properties. If DMProperties contains table properties, then + // table properties of datamap table will be updated + if (parentTablesList.size() == 1) { + DataMapUtil + .inheritTablePropertiesFromMainTable( + parentTablesList.get(0), + fields, + fieldRelationMap, + tableProperties) + if (granularity != null) { + val timeSeriesDataType = parentTablesList + .get(0) + .getTableInfo + .getFactTable + .getListOfColumns + .asScala + .filter(columnSchema => columnSchema.getColumnName + .equalsIgnoreCase(timeSeriesColumn)) + .head + .getDataType + if (timeSeriesDataType.equals(DataTypes.DATE) || + timeSeriesDataType.equals(DataTypes.TIMESTAMP)) { + // if data type is of Date type, then check if given granularity is valid for date type + if (timeSeriesDataType.equals(DataTypes.DATE)) { + TimeSeriesUtil.validateTimeSeriesGranularityForDate(granularity) + } + } else { + throw new MalformedCarbonCommandException( + "TimeSeries Column must be of TimeStamp or Date type") + } + } + } + dmProperties.foreach(t => tableProperties.put(t._1, t._2)) + val usePartitioning = dmProperties.getOrElse("partitioning", "true").toBoolean + var partitionerFields: Seq[PartitionerField] = Seq.empty + // Inherit partition from parent table if datamap is mapped to single parent table + if (parentTablesList.size() == 1) { + val partitionInfo = parentTablesList.get(0).getPartitionInfo + val parentPartitionColumns = if (!usePartitioning) { + Seq.empty + } else if (parentTablesList.get(0).isHivePartitionTable) { + partitionInfo.getColumnSchemaList.asScala.map(_.getColumnName) + } else { + Seq() + } + partitionerFields = PartitionUtils + .getPartitionerFields(parentPartitionColumns, fieldRelationMap) + } + + var order = 0 + val columnOrderMap = new java.util.HashMap[Integer, String]() + if (partitionerFields.nonEmpty) { + fields.foreach { field => + columnOrderMap.put(order, field.column) + order += 1 + } + } + + // TODO Use a proper DB + val tableIdentifier = + TableIdentifier(dataMapSchema.getDataMapName + "_table", + selectTables.head.identifier.database) + // prepare table model of the collected tokens + val tableModel: TableModel = CarbonParserUtil.prepareTableModel( + ifNotExistPresent = ifNotExistsSet, + CarbonParserUtil.convertDbNameToLowerCase(tableIdentifier.database), + tableIdentifier.table.toLowerCase, + fields, + partitionerFields, + tableProperties, + None, + isAlterFlow = false, + None) + + val tablePath = if (dmProperties.contains("path")) { + dmProperties("path") + } else { + CarbonEnv.getTablePath(tableModel.databaseNameOp, tableModel.tableName)(sparkSession) + } + CarbonCreateTableCommand(TableNewProcessor(tableModel), + tableModel.ifNotExistsSet, Some(tablePath), isVisible = false).run(sparkSession) + + // Map list of main table columns mapped to datamap table and add to dataMapSchema + val mainTableToColumnsMap = new java.util.HashMap[String, util.Set[String]]() + val mainTableFieldIterator = fieldRelationMap.values.asJava.iterator() + while (mainTableFieldIterator.hasNext) { + val value = mainTableFieldIterator.next() + value.columnTableRelationList.foreach { + columnTableRelation => + columnTableRelation.foreach { + mainTable => + if (null == mainTableToColumnsMap.get(mainTable.parentTableName)) { + val columns = new util.HashSet[String]() + columns.add(mainTable.parentColumnName.toLowerCase()) + mainTableToColumnsMap.put(mainTable.parentTableName, columns) + } else { + mainTableToColumnsMap.get(mainTable.parentTableName) + .add(mainTable.parentColumnName.toLowerCase()) + } + } + } + } + dataMapSchema.setMainTableColumnList(mainTableToColumnsMap) + dataMapSchema.setColumnsOrderMap(columnOrderMap) + if (null != granularity && null != timeSeriesColumn) { + dataMapSchema.setCtasQuery(queryString) + dataMapSchema.setTimeSeries(true) + } else { + dataMapSchema.setCtasQuery(updatedQueryWithDb) + } + dataMapSchema + .setRelationIdentifier(new RelationIdentifier(tableIdentifier.database.get, + tableIdentifier.table, + CarbonEnv.getCarbonTable(tableIdentifier)(sparkSession).getTableId)) + + val parentIdents = selectTables.map { table => + val relationIdentifier = new RelationIdentifier(table.database, table.identifier.table, "") + relationIdentifier.setTablePath(FileFactory.getUpdatedFilePath(table.location.toString)) + relationIdentifier + } + dataMapSchema.getRelationIdentifier.setTablePath(tablePath) + dataMapSchema.setParentTables(new util.ArrayList[RelationIdentifier](parentIdents.asJava)) + dataMapSchema.getProperties.put(DataMapProperty.FULL_REFRESH, fullRebuild.toString) + try { + DataMapStoreManager.getInstance().saveDataMapSchema(dataMapSchema) + } catch { + case ex: Exception => + val dropTableCommand = CarbonDropTableCommand(true, + new Some[String](dataMapSchema.getRelationIdentifier.getDatabaseName), + dataMapSchema.getRelationIdentifier.getTableName, + true) + dropTableCommand.run(sparkSession) + throw ex + } + } + + private def validateMVQuery( + sparkSession: SparkSession, + logicalPlan: LogicalPlan): ModularPlan = { + val dataMapProvider = DataMapManager.get().getDataMapProvider(null, + new DataMapSchema("", DataMapClassProvider.MV.getShortName), sparkSession) + var catalog = DataMapStoreManager.getInstance().getDataMapCatalog(dataMapProvider, + DataMapClassProvider.MV.getShortName).asInstanceOf[SummaryDatasetCatalog] + if (catalog == null) { + catalog = new SummaryDatasetCatalog(sparkSession) + } + val modularPlan = + catalog.mvSession.sessionState.modularizer.modularize( + catalog.mvSession.sessionState.optimizer.execute(logicalPlan)).next().semiHarmonized + + // Only queries which can be select , predicate , join, group by and having queries. + if (!modularPlan.isSPJGH) { + throw new UnsupportedOperationException("MV is not supported for this query") + } + val isValid = modularPlan match { + case g: GroupBy => + // Make sure all predicates are present in projections. + g.predicateList.forall{p => + g.outputList.exists{ + case a: Alias => + a.semanticEquals(p) || a.child.semanticEquals(p) + case other => other.semanticEquals(p) + } + } + case _ => true + } + if (!isValid) { + throw new UnsupportedOperationException( + "Group by columns must be present in project columns") + } + if (catalog.isMVWithSameQueryPresent(logicalPlan)) { + throw new UnsupportedOperationException("MV with same query present") + } + + var expressionValid = true + modularPlan.transformExpressions { + case coal@Coalesce(_) if coal.children.exists( + exp => exp.isInstanceOf[AggregateExpression]) => + expressionValid = false + coal + } + + if (!expressionValid) { + throw new UnsupportedOperationException("MV doesn't support Coalesce") + } + modularPlan + } + + def getUpdatedName(name: String, counter: Int): String = { + var updatedName = name.replace("(", "_") + .replace(")", "") + .replace(" ", "_") + .replace("=", "") + .replace(",", "") + .replace(".", "_") + .replace("`", "") + if (updatedName.length >= CarbonCommonConstants.MAXIMUM_CHAR_LENGTH) { + updatedName = updatedName.substring(0, 110) + CarbonCommonConstants.UNDERSCORE + counter + } + updatedName + } + + private def updateColumnName(attr: Attribute, counter: Int): String = { + val name = getUpdatedName(attr.name, counter) + val value = attr.qualifier.map(qualifier => qualifier + "_" + name) Review comment: use constants from CarbonCommonConstant ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378981003 ########## File path: datamap/mv/core/src/main/spark2.3/org/apache/carbondata/mv/extension/MVRules.scala ########## @@ -0,0 +1,51 @@ +/* + * 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.mv.extension + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.SessionCatalog +import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.hive.CarbonMVRules + +class MVRules( Review comment: since two classes MVRules in 2.3 and 2.4 are same, can we keep in common place itself, instead of two classes? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378971907 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/command/CreateMaterializedViewCommand.scala ########## @@ -0,0 +1,106 @@ +/* + * 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.mv.extension.command + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.command._ + +import org.apache.carbondata.common.exceptions.sql.MalformedMaterializedViewException +import org.apache.carbondata.common.logging.LogServiceFactory +import org.apache.carbondata.core.datamap.{DataMapProvider, DataMapStoreManager} +import org.apache.carbondata.core.datamap.status.DataMapStatusManager +import org.apache.carbondata.core.metadata.schema.datamap.DataMapProperty +import org.apache.carbondata.core.metadata.schema.table.DataMapSchema +import org.apache.carbondata.core.util.CarbonProperties +import org.apache.carbondata.datamap.DataMapManager +import org.apache.carbondata.events._ +import org.apache.carbondata.mv.extension.MVDataMapProvider + +/** + * Create Materialized View Command implementation + * It will create the MV table, load the MV table (if deferred rebuild is false), + * and register the MV schema in [[DataMapStoreManager]] + */ +case class CreateMaterializedViewCommand( + mvName: String, + properties: Map[String, String], + queryString: Option[String], + ifNotExistsSet: Boolean = false, + deferredRebuild: Boolean = false) + extends AtomicRunnableCommand { + + private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) + private var dataMapProvider: DataMapProvider = _ + private var dataMapSchema: DataMapSchema = _ + + override def processMetadata(sparkSession: SparkSession): Seq[Row] = { + + setAuditInfo(Map("mvName" -> mvName) ++ properties) + + dataMapSchema = new DataMapSchema(mvName, MVDataMapProvider.MV_PROVIDER_NAME) + val property = properties.map(x => (x._1.trim, x._2.trim)).asJava Review comment: rename to datamap properties ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378981200 ########## File path: datamap/mv/core/src/test/scala/org/apache/carbondata/mv/plans/IsSPJGHSuite.scala ########## @@ -20,6 +20,7 @@ package org.apache.carbondata.mv.plans import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation + Review comment: revert the change if not reqired ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378981536 ########## File path: datamap/mv/core/src/test/scala/org/apache/carbondata/mv/plans/LogicalToModularPlanSuite.scala ########## @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.aggregate.Count import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, _} + Review comment: same as above for imports ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378966804 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVExtensionSqlParser.scala ########## @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.carbondata.mv.extension + +import org.apache.spark.sql.{CarbonEnv, CarbonUtils, SparkSession} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkSqlParser +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.CarbonException + +import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException +import org.apache.carbondata.spark.util.CarbonScalaUtil + +/** + * Parser for Materialized View related command + */ +class MVExtensionSqlParser( + conf: SQLConf, + sparkSession: SparkSession, + initialParser: ParserInterface +) extends SparkSqlParser(conf) { + + val parser = new MVParser + + override def parsePlan(sqlText: String): LogicalPlan = { + parser.synchronized { + CarbonEnv.getInstance(sparkSession) + } + CarbonUtils.updateSessionInfoToCurrentThread(sparkSession) + try { + val plan = parser.parse(sqlText) + plan + } catch { + case ce: MalformedCarbonCommandException => + throw ce + case ex: Throwable => + try { + val parsedPlan = initialParser.parsePlan(sqlText) + CarbonScalaUtil.cleanParserThreadLocals + parsedPlan + } catch { + case mce: MalformedCarbonCommandException => + throw mce + case e: Throwable => + e.printStackTrace(System.err) Review comment: no need to print complete stackTrace i think ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
akashrn5 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r378976973 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/command/RebuildMaterializedViewCommand.scala ########## @@ -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.mv.extension.command + +import org.apache.spark.sql.{CarbonEnv, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.command.DataCommand + +import org.apache.carbondata.common.exceptions.sql.MalformedMaterializedViewException +import org.apache.carbondata.core.datamap.DataMapStoreManager +import org.apache.carbondata.core.datamap.status.DataMapStatusManager +import org.apache.carbondata.core.util.CarbonProperties +import org.apache.carbondata.datamap.DataMapManager +import org.apache.carbondata.events.{UpdateDataMapPostExecutionEvent, _} + +/** + * Refresh Materialized View Command implementation + * This command refresh the MV table incrementally and make it synchronized with the main + * table. After sync, MV state is changed to enabled. + */ +case class RebuildMaterializedViewCommand( + mvName: String) extends DataCommand { + + override def processData(sparkSession: SparkSession): Seq[Row] = { + import scala.collection.JavaConverters._ + val schemas = DataMapStoreManager.getInstance().getAllDataMapSchemas + val schemaOption = schemas.asScala.find(p => p.getDataMapName.equalsIgnoreCase(mvName)) + if (schemaOption.isEmpty) { + throw new MalformedMaterializedViewException(s"Materialized view $mvName does not exist") + } + val schema = schemaOption.get Review comment: rename to `datamapSchema` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
Indhumathi27 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r379252982 ########## File path: datamap/mv/core/src/main/scala/org/apache/carbondata/mv/extension/command/ShowMaterializedViewCommand.scala ########## @@ -0,0 +1,97 @@ +/* + * 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.mv.extension.command + +import java.util + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.{CarbonEnv, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.execution.command.{Checker, DataCommand} +import org.apache.spark.sql.types.{BooleanType, StringType} + +import org.apache.carbondata.core.datamap.DataMapStoreManager +import org.apache.carbondata.core.metadata.schema.table.DataMapSchema +import org.apache.carbondata.mv.extension.MVDataMapProvider + +/** + * Show Materialized View Command implementation + * + */ +case class ShowMaterializedViewCommand(tableIdentifier: Option[TableIdentifier]) + extends DataCommand { + + override def output: Seq[Attribute] = { + Seq( + AttributeReference("Name", StringType, nullable = false)(), + AttributeReference("Associated Table", StringType, nullable = false)(), + AttributeReference("Refresh", StringType, nullable = false)(), + AttributeReference("Incremental", BooleanType, nullable = false)(), + AttributeReference("Properties", StringType, nullable = false)(), + AttributeReference("Status", StringType, nullable = false)(), + AttributeReference("Sync Info", StringType, nullable = false)()) + } + + override def processData(sparkSession: SparkSession): Seq[Row] = { + convertToRow(getAllMVSchema(sparkSession)) + } + + /** + * get all datamaps for this table, including preagg, index datamaps and mv Review comment: please change the description only for mv ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
In reply to this post by GitBox
Indhumathi27 commented on a change in pull request #3612: [CARBONDATA-3694] Separate Materialized View command from DataMap command
URL: https://github.com/apache/carbondata/pull/3612#discussion_r379264939 ########## File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DataMapSchema.java ########## @@ -290,4 +301,71 @@ public boolean isTimeSeries() { public void setTimeSeries(boolean timeSeries) { isTimeSeries = timeSeries; } + + public boolean supportIncrementalBuild() { + String prop = getProperties().get(DataMapProperty.FULL_REFRESH); + return prop == null || prop.equalsIgnoreCase("false"); + } + + public String getPropertiesAsString() { + String[] properties = getProperties().entrySet().stream() + // ignore internal used property + .filter(p -> + !p.getKey().equalsIgnoreCase(DataMapProperty.DEFERRED_REBUILD) && + !p.getKey().equalsIgnoreCase(DataMapProperty.CHILD_SELECT_QUERY) && Review comment: `DataMapProperty.CHILD_SELECT_QUERY` && `DataMapProperty.QUERY_TYPE` was used for preaggregate. Please remove it. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [hidden email] With regards, Apache Git Services |
Free forum by Nabble | Edit this page |