ravipesala commented on a change in pull request #3483: [CARBONDATA-3597] Support Merge for SCD and CCD scenarios
URL: https://github.com/apache/carbondata/pull/3483#discussion_r361897551 ########## File path: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetCommand.scala ########## @@ -0,0 +1,520 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.command.mutation.merge + +import java.util.UUID + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.spark.sql.{AnalysisException, CarbonDatasourceHadoopRelation, Column, DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.command.{DataCommand, ExecutionErrors} +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.DistributionUtil +import org.apache.spark.sql.types.{IntegerType, StringType, StructField} +import org.apache.spark.sql.util.SparkSQLUtil +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, LongAccumulator} + +import org.apache.carbondata.common.logging.LogServiceFactory +import org.apache.carbondata.core.constants.CarbonCommonConstants +import org.apache.carbondata.core.datamap.Segment +import org.apache.carbondata.core.metadata.SegmentFileStore +import org.apache.carbondata.core.metadata.schema.table.CarbonTable +import org.apache.carbondata.core.mutate.CarbonUpdateUtil +import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus} +import org.apache.carbondata.core.util.path.CarbonTablePath +import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat +import org.apache.carbondata.hadoop.internal.ObjectArrayWritable +import org.apache.carbondata.processing.loading.FailureCauses +import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder} +import org.apache.carbondata.processing.util.CarbonLoaderUtil + +/** + * This command will merge the data of source dataset to target dataset backed by carbon table. + * @param targetDsOri Target dataset to merge the data. This dataset should be backed by carbontable + * @param srcDS Source dataset, it can be any data. + * @param mergeMatches It contains the join condition and list match conditions to apply. + */ +case class CarbonMergeDataSetCommand(targetDsOri: Dataset[Row], + srcDS: Dataset[Row], + var mergeMatches: MergeDataSetMatches) + extends DataCommand { + + val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) + + override def processData(sparkSession: SparkSession): Seq[Row] = { + val rltn = collectCarbonRelation(targetDsOri.logicalPlan) + // Target dataset must be backed by carbondata table. + if (rltn.length != 1) { + throw new UnsupportedOperationException( + "Carbon table supposed to be present in merge dataset") + } + // validate the merge matches and actions. + validateMergeActions(mergeMatches, targetDsOri, sparkSession) + val carbonTable = rltn.head.carbonRelation.carbonTable + val hasDelAction = mergeMatches.matchList + .exists(_.getActions.exists(_.isInstanceOf[DeleteAction])) + val hasUpdateAction = mergeMatches.matchList + .exists(_.getActions.exists(_.isInstanceOf[UpdateAction])) + val (insertHistOfUpdate, insertHistOfDelete) = getInsertHistoryStatus(mergeMatches) + // Get all the required columns of targetDS by going through all match conditions and actions. + val columns = getSelectExpressionsOnExistingDF(targetDsOri, mergeMatches, sparkSession) + // select only the required columns, it can avoid lot of and shuffling. + val targetDs = targetDsOri.select(columns: _*) + // Update the update mapping with unfilled columns.From here on system assumes all mappings + // are existed. + mergeMatches = updateMappingIfNotExists(mergeMatches, targetDs) + // Lets generate all conditions combinations as one column and add them as 'status'. + val condition = generateStatusColumnWithAllCombinations(mergeMatches) + + // Add the tupleid udf to get the tupleid to generate delete delta. + val frame = targetDs.withColumn(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID, + expr("getTupleId()")).withColumn("exist_on_target", lit(1)).join( + srcDS.withColumn("exist_on_src", lit(1)), + // Do the full outer join to get the data from both sides without missing anything. + // TODO As per the match conditions choose the join, sometimes it might be possible to use + // left_outer join. + mergeMatches.joinExpr, "full_outer").withColumn("status", condition) + if (LOGGER.isDebugEnabled) { + frame.explain() + } + val tableCols = + carbonTable.getTableInfo.getFactTable.getListOfColumns.asScala.map(_.getColumnName). + filterNot(_.equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)) + val builder = new CarbonLoadModelBuilder(carbonTable) + val options = Seq(("fileheader", tableCols.mkString(","))).toMap + val model = builder.build(options.asJava, CarbonUpdateUtil.readCurrentTime, "1") + model.setLoadWithoutConverterStep(true) + val newLoadMetaEntry = new LoadMetadataDetails + CarbonLoaderUtil.populateNewLoadMetaEntry(newLoadMetaEntry, + SegmentStatus.INSERT_IN_PROGRESS, + model.getFactTimeStamp, + false) + CarbonLoaderUtil.recordNewLoadMetadata(newLoadMetaEntry, model, true, false) + + model.setCsvHeader(tableCols.mkString(",")) + + val projections: Seq[Seq[MergeProjection]] = mergeMatches.matchList.map { m => + m.getActions.map { + case u: UpdateAction => MergeProjection(tableCols, frame, rltn.head, sparkSession, u) + case i: InsertAction => MergeProjection(tableCols, frame, rltn.head, sparkSession, i) + case d: DeleteAction => MergeProjection(tableCols, frame, rltn.head, sparkSession, d) + case _ => null + }.filter(_ != null) + } + + val st = System.currentTimeMillis() + // Create accumulators to log the stats + val stats = Stats(createLongAccumalator("insertedRows"), + createLongAccumalator("updatedRows"), + createLongAccumalator("deletedRows")) + val processedRDD = processIUD(sparkSession, frame, carbonTable, model, projections, stats) + + val executorErrors = ExecutionErrors(FailureCauses.NONE, "") + val trxMgr = TranxManager(model.getFactTimeStamp) + + val mutationAction = MutationActionFactory.getMutationAction(sparkSession, + carbonTable, hasDelAction, hasUpdateAction, + insertHistOfUpdate, insertHistOfDelete) + + val tuple = mutationAction.handleAction(processedRDD, executorErrors, trxMgr) + + // In case user only has insert action. + if (!(hasDelAction || hasUpdateAction)) { + processedRDD.count() + } + LOGGER.info(s"Total inserted rows: ${stats.insertedRows.sum}") + LOGGER.info(s"Total updated rows: ${stats.updatedRows.sum}") + LOGGER.info(s"Total deleted rows: ${stats.deletedRows.sum}") + LOGGER.info( + " Time taken to merge data : " + tuple + " :: " + (System.currentTimeMillis() - st)) + + val segment = new Segment(model.getSegmentId, + SegmentFileStore.genSegmentFileName( + model.getSegmentId, + System.nanoTime().toString) + CarbonTablePath.SEGMENT_EXT, + CarbonTablePath.getSegmentPath(carbonTable.getTablePath, + model.getSegmentId), Map.empty[String, String].asJava) + val writeSegment = + SegmentFileStore.writeSegmentFile(carbonTable, segment) + + if (writeSegment) { + SegmentFileStore.updateTableStatusFile( + carbonTable, + model.getSegmentId, + segment.getSegmentFileName, + carbonTable.getCarbonTableIdentifier.getTableId, + new SegmentFileStore(carbonTable.getTablePath, segment.getSegmentFileName), + SegmentStatus.SUCCESS) + } else { + CarbonLoaderUtil.updateTableStatusForFailure(model) + } + + if (hasDelAction || hasUpdateAction) { + if (CarbonUpdateUtil.updateSegmentStatus(tuple._1, carbonTable, + trxMgr.getLatestTrx.toString, false) && + CarbonUpdateUtil + .updateTableMetadataStatus( + model.getLoadMetadataDetails.asScala.map(l => + new Segment(l.getMergedLoadName, + l.getSegmentFile)).toSet.asJava, + carbonTable, + trxMgr.getLatestTrx.toString, + true, + tuple._2.asJava)) { + LOGGER.info(s"Merge data operation is successful for " + + s"${ carbonTable.getDatabaseName }.${ carbonTable.getTableName }") + } else { + throw new CarbonMergeDataSetException("Saving update status or table status failed") + } + } + // Load the history table if the inserthistorytable action is added by user. + HistoryTableLoadHelper.loadHistoryTable(sparkSession, rltn.head, carbonTable, + trxMgr, mutationAction, mergeMatches) + Seq.empty + } + + private def processIUD(sparkSession: SparkSession, + frame: DataFrame, + carbonTable: CarbonTable, + model: CarbonLoadModel, + projections: Seq[Seq[MergeProjection]], + stats: Stats) = { + val conf = SparkSQLUtil.sessionState(sparkSession).newHadoopConf() Review comment: ok ---------------------------------------------------------------- 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 |