YannByron commented on code in PR #6999: URL: https://github.com/apache/hudi/pull/6999#discussion_r1002869098
########## hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala: ########## @@ -0,0 +1,221 @@ +/* + * 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.hudi.command + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} + +import org.apache.hudi.common.table.HoodieTableConfig + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.execution.command.PartitionStatistics +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} + +import java.util.concurrent.TimeUnit.MILLISECONDS + +import scala.util.control.NonFatal + +/** + * Command for repair hudi table's partitions. + * Use hoodieCatalogTable.getPartitionPaths() to get partitions instead of scanning the file system. + */ +case class RepairHoodieTableCommand(tableName: TableIdentifier, + enableAddPartitions: Boolean, + enableDropPartitions: Boolean, + cmd: String = "MSCK REPAIR TABLE") extends HoodieLeafRunnableCommand { + + // These are list of statistics that can be collected quickly without requiring a scan of the data + // see https://github.com/apache/hive/blob/master/ + // common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java + val NUM_FILES = "numFiles" + val TOTAL_SIZE = "totalSize" + val DDL_TIME = "transient_lastDdlTime" + + private def getPathFilter(hadoopConf: Configuration): PathFilter = { + // Dummy jobconf to get to the pathFilter defined in configuration + // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) + val jobConf = new JobConf(hadoopConf, this.getClass) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + new PathFilter { + override def accept(path: Path): Boolean = { + val name = path.getName + if (name != "_SUCCESS" && name != "_temporary" && !name.startsWith(".")) { + pathFilter == null || pathFilter.accept(path) + } else { + false + } + } + } + } + + override def run(spark: SparkSession): Seq[Row] = { + val catalog = spark.sessionState.catalog + val table = catalog.getTableMetadata(tableName) + val tableIdentWithDB = table.identifier.quotedString + if (table.partitionColumnNames.isEmpty) { + throw new AnalysisException( + s"Operation not allowed: $cmd only works on partitioned tables: $tableIdentWithDB") + } + + if (table.storage.locationUri.isEmpty) { + throw new AnalysisException(s"Operation not allowed: $cmd only works on table with " + + s"location provided: $tableIdentWithDB") + } + + val root = new Path(table.location) + logInfo(s"Recover all the partitions in $root") + + val hoodieCatalogTable = HoodieCatalogTable(spark, table.identifier) + val isHiveStyledPartitioning = hoodieCatalogTable.catalogProperties. + getOrElse(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key, "true").equals("true") + val partitionSpecsAndLocs: Seq[(TablePartitionSpec, Path)] = hoodieCatalogTable.getPartitionPaths.map(partitionPath => { + var values = partitionPath.split('/') + if (isHiveStyledPartitioning) { + values = values.map(_.split('=')(1)) + } + (table.partitionColumnNames.zip(values).toMap, new Path(root, partitionPath)) + }) + + val droppedAmount = if (enableDropPartitions) { + dropPartitions(catalog, partitionSpecsAndLocs) + } else 0 + val addedAmount = if (enableAddPartitions) { + val hadoopConf = spark.sessionState.newHadoopConf() + val fs = root.getFileSystem(hadoopConf) + val pathFilter = getPathFilter(hadoopConf) + val threshold = spark.sparkContext.conf.getInt("spark.rdd.parallelListingThreshold", 10) + val total = partitionSpecsAndLocs.length + val partitionStats = if (spark.sqlContext.conf.gatherFastStats) { + gatherPartitionStats(spark, partitionSpecsAndLocs, fs, pathFilter, threshold) + } else { + Map.empty[String, PartitionStatistics] + } + logInfo(s"Finished to gather the fast stats for all $total partitions.") + addPartitions(spark, table, partitionSpecsAndLocs, partitionStats) + total + } else 0 + // Updates the table to indicate that its partition metadata is stored in the Hive metastore. + // This is always the case for Hive format tables, but is not true for Datasource tables created + // before Spark 2.1 unless they are converted via `msck repair table`. + spark.sessionState.catalog.alterTable(table.copy(tracksPartitionsInCatalog = true)) + try { + spark.catalog.refreshTable(tableIdentWithDB) + } catch { + case NonFatal(e) => + logError(s"Cannot refresh the table '$tableIdentWithDB'. A query of the table " + + "might return wrong result if the table was cached. To avoid such issue, you should " + + "uncache the table manually via the UNCACHE TABLE command after table recovering will " + + "complete fully.", e) + } + logInfo(s"Recovered all partitions: added ($addedAmount), dropped ($droppedAmount).") + Seq.empty[Row] + } + + private def gatherPartitionStats(spark: SparkSession, Review Comment: This method is not suitable for Hudi too. Gather the stat of the current snapshot, not the whole files in FileSystem. -- 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
