From 6b6273dc0ab52781e3b7b905d0b78824605a4e64 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Fri, 4 Nov 2016 19:03:57 -0700 Subject: [PATCH 01/27] Fri Nov 4 19:03:57 PDT 2016 --- .../sql/catalyst/parser/AstBuilder.scala | 9 +-- .../plans/logical/basicLogicalOperators.scala | 10 ++-- .../sql/catalyst/parser/PlanParserSuite.scala | 4 +- .../execution/datasources/DataSource.scala | 2 + .../datasources/DataSourceStrategy.scala | 60 ++++++++++++------- .../InsertIntoHadoopFsRelationCommand.scala | 23 ++++++- .../datasources/PartitioningUtils.scala | 6 ++ .../PartitionProviderCompatibilitySuite.scala | 2 + 8 files changed, 82 insertions(+), 34 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 4b151c81d8f8..072522ce3f23 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -178,18 +178,13 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx) } val overwrite = ctx.OVERWRITE != null - val overwritePartition = - if (overwrite && partitionKeys.nonEmpty && dynamicPartitionKeys.isEmpty) { - Some(partitionKeys.map(t => (t._1, t._2.get))) - } else { - None - } + val staticPartitionKeys = partitionKeys.filter(_._2.nonEmpty).map(t => (t._1, t._2.get)) InsertIntoTable( UnresolvedRelation(tableIdent, None), partitionKeys, query, - OverwriteOptions(overwrite, overwritePartition), + OverwriteOptions(overwrite, staticPartitionKeys), ctx.EXISTS != null) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 65ceab2ce27b..574caf039d3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -350,13 +350,15 @@ case class BroadcastHint(child: LogicalPlan) extends UnaryNode { * Options for writing new data into a table. * * @param enabled whether to overwrite existing data in the table. - * @param specificPartition only data in the specified partition will be overwritten. + * @param staticPartitionKeys if non-empty, specifies that we only want to overwrite partitions + * that match this partial partition spec. If empty, all partitions + * will be overwritten. */ case class OverwriteOptions( enabled: Boolean, - specificPartition: Option[CatalogTypes.TablePartitionSpec] = None) { - if (specificPartition.isDefined) { - assert(enabled, "Overwrite must be enabled when specifying a partition to overwrite.") + staticPartitionKeys: CatalogTypes.TablePartitionSpec = Map.empty) { + if (staticPartitionKeys.nonEmpty) { + assert(enabled, "Overwrite must be enabled when specifying specific partitions.") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 7400f3430e99..e5f1f7b3bd4c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -185,9 +185,9 @@ class PlanParserSuite extends PlanTest { OverwriteOptions( overwrite, if (overwrite && partition.nonEmpty) { - Some(partition.map(kv => (kv._1, kv._2.get))) + partition.map(kv => (kv._1, kv._2.get)) } else { - None + Map.empty }), ifNotExists) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 52666119351b..a4c323595a0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -418,6 +418,8 @@ case class DataSource( val plan = InsertIntoHadoopFsRelationCommand( outputPath, + Map.empty, + Map.empty, columns, bucketSpec, format, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index a548e88cb683..eff2026f7fac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SimpleCatalogRelation} @@ -146,6 +146,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { // values in the PARTITION clause (e.g. b in the above example). // dynamic_partitioning_columns are partitioning columns that do not assigned // values in the PARTITION clause (e.g. c in the above example). + // TODO(ekl) get rid of this and combine with static partition key arg for overwrite case insert @ logical.InsertIntoTable( relation @ LogicalRelation(t: HadoopFsRelation, _, _), parts, query, overwrite, false) if query.resolved && parts.exists(_._2.isDefined) => @@ -182,25 +183,19 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { "Cannot overwrite a path that is also being read from.") } - val overwritingSinglePartition = - overwrite.specificPartition.isDefined && - t.sparkSession.sessionState.conf.manageFilesourcePartitions && - l.catalogTable.get.tracksPartitionsInCatalog - - val effectiveOutputPath = if (overwritingSinglePartition) { - val partition = t.sparkSession.sessionState.catalog.getPartition( - l.catalogTable.get.identifier, overwrite.specificPartition.get) - new Path(partition.storage.locationUri.get) - } else { - outputPath - } + val partitionLocationOverrides: Map[String, String] = + if (t.sparkSession.sessionState.conf.manageFilesourcePartitions && + l.catalogTable.get.tracksPartitionsInCatalog) { + getPartitionLocationOverrides( + t.sparkSession, l.catalogTable.get, outputPath, overwrite.staticPartitionKeys) + } else { + Map.empty + } - val effectivePartitionSchema = if (overwritingSinglePartition) { - Nil - } else { - query.resolve(t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver) - } + val partitionSchema = query.resolve( + t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver) + // TODO(ekl) move this into InsertIntoHadoopFsRelationCommand def refreshPartitionsCallback(updatedPartitions: Seq[TablePartitionSpec]): Unit = { if (l.catalogTable.isDefined && updatedPartitions.nonEmpty && l.catalogTable.get.partitionColumnNames.nonEmpty && @@ -215,8 +210,10 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { } val insertCmd = InsertIntoHadoopFsRelationCommand( - effectiveOutputPath, - effectivePartitionSchema, + outputPath, + if (overwrite.enabled) overwrite.staticPartitionKeys else Map.empty, + partitionLocationOverrides, + partitionSchema, t.bucketSpec, t.fileFormat, refreshPartitionsCallback, @@ -226,6 +223,29 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { insertCmd } + + private def getPartitionLocationOverrides( + spark: SparkSession, + table: CatalogTable, + basePath: Path, + staticPartitionKeys: TablePartitionSpec): Map[String, String] = { + val hadoopConf = spark.sessionState.newHadoopConf + val fs = basePath.getFileSystem(hadoopConf) + val qualifiedBasePath = basePath.makeQualified(fs.getUri, fs.getWorkingDirectory) + val partitions = spark.sessionState.catalog.listPartitions( + table.identifier, Some(staticPartitionKeys)) + partitions.flatMap { p => + val defaultLocation = qualifiedBasePath.suffix( + "/" + PartitioningUtils.getPathFragment(p.spec, table.partitionSchema)).toString + val catalogLocation = new Path(p.storage.locationUri.get).makeQualified( + fs.getUri, fs.getWorkingDirectory).toString + if (catalogLocation != defaultLocation) { + Some(defaultLocation -> catalogLocation) + } else { + None + } + }.toMap + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 9c75e2ae7476..a271a65a8a69 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -35,6 +35,8 @@ import org.apache.spark.sql.execution.command.RunnableCommand */ case class InsertIntoHadoopFsRelationCommand( outputPath: Path, + staticPartitionKeys: TablePartitionSpec, + partitionLocationOverrides: Map[String, String], partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], fileFormat: FileFormat, @@ -44,6 +46,9 @@ case class InsertIntoHadoopFsRelationCommand( mode: SaveMode) extends RunnableCommand { + println("static partition keys: " + staticPartitionKeys) + println("overrides: " + partitionLocationOverrides) + override protected def innerChildren: Seq[LogicalPlan] = query :: Nil override def run(sparkSession: SparkSession): Seq[Row] = { @@ -65,7 +70,23 @@ case class InsertIntoHadoopFsRelationCommand( case (SaveMode.ErrorIfExists, true) => throw new AnalysisException(s"path $qualifiedOutputPath already exists.") case (SaveMode.Overwrite, true) => - if (!fs.delete(qualifiedOutputPath, true /* recursively */)) { + val suffix = if (staticPartitionKeys.nonEmpty) { + "/" + partitionColumns.flatMap { p => + staticPartitionKeys.get(p.name) match { + case Some(value) => + Some( + PartitioningUtils.escapePathName(p.name) + "=" + + PartitioningUtils.escapePathName(value)) + case None => + None + } + }.mkString("/") + } else { + "" + } + val pathToDelete = qualifiedOutputPath.suffix(suffix) + println("path to delete: " + pathToDelete) + if (fs.exists(pathToDelete) && !fs.delete(pathToDelete, true /* recursively */)) { throw new IOException(s"Unable to clear output " + s"directory $qualifiedOutputPath prior to writing to it") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index b51b41869bf0..464d37c32907 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -256,6 +256,12 @@ object PartitioningUtils { }.toMap } + def getPathFragment(spec: TablePartitionSpec, partitionSchema: StructType): String = { + partitionSchema.map { field => + escapePathName(field.name) + "=" + escapePathName(spec(field.name)) + }.mkString("/") + } + /** * Normalize the column names in partition specification, w.r.t. the real partition column names * and case sensitivity. e.g., if the partition spec has a column named `monTh`, and there is a diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index ac435bf6195b..14ecfab291ab 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -60,6 +60,7 @@ class PartitionProviderCompatibilitySuite } } +/* test("convert partition provider to hive with repair table") { withTable("test") { withTempDir { dir => @@ -154,6 +155,7 @@ class PartitionProviderCompatibilitySuite } } +*/ test("insert overwrite partition of new datasource table overwrites just partition") { withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { withTable("test") { From 40f436850c0339e0ca02fa7f542305b815c38ab8 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Fri, 4 Nov 2016 19:42:34 -0700 Subject: [PATCH 02/27] Fri Nov 4 19:42:34 PDT 2016 --- .../execution/datasources/FileFormatWriter.scala | 13 +++++++++---- .../InsertIntoHadoopFsRelationCommand.scala | 7 ++++--- .../sql/execution/streaming/FileStreamSink.scala | 2 +- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index e404dcd5452b..ecf07a8cb0fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -47,6 +47,9 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter /** A helper object for writing FileFormat data out to a location. */ object FileFormatWriter extends Logging { + /** Describes how output files should be placed in the filesystem. */ + case class OutputSpec(outputPath: String, customPartitionLocations: Map[String, String]) + /** A shared job description for all the write tasks. */ private class WriteJobDescription( val uuid: String, // prevent collision between different (appending) write jobs @@ -56,7 +59,8 @@ object FileFormatWriter extends Logging { val partitionColumns: Seq[Attribute], val nonPartitionColumns: Seq[Attribute], val bucketSpec: Option[BucketSpec], - val path: String) + val path: String, + val customPartitionLocations: Map[String, String]) extends Serializable { assert(AttributeSet(allColumns) == AttributeSet(partitionColumns ++ nonPartitionColumns), @@ -83,7 +87,7 @@ object FileFormatWriter extends Logging { plan: LogicalPlan, fileFormat: FileFormat, committer: FileCommitProtocol, - outputPath: String, + outputSpec: OutputSpec, hadoopConf: Configuration, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], @@ -93,7 +97,7 @@ object FileFormatWriter extends Logging { val job = Job.getInstance(hadoopConf) job.setOutputKeyClass(classOf[Void]) job.setOutputValueClass(classOf[InternalRow]) - FileOutputFormat.setOutputPath(job, new Path(outputPath)) + FileOutputFormat.setOutputPath(job, new Path(outputSpec.outputPath)) val partitionSet = AttributeSet(partitionColumns) val dataColumns = plan.output.filterNot(partitionSet.contains) @@ -111,7 +115,8 @@ object FileFormatWriter extends Logging { partitionColumns = partitionColumns, nonPartitionColumns = dataColumns, bucketSpec = bucketSpec, - path = outputPath) + path = outputSpec.outputPath, + customPartitionLocations = outputSpec.customPartitionLocations) SQLExecution.withNewExecutionId(sparkSession, queryExecution) { // This call shouldn't be put into the `try` block below because it only initializes and diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index a271a65a8a69..a3d2cbafd0b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution.command.RunnableCommand case class InsertIntoHadoopFsRelationCommand( outputPath: Path, staticPartitionKeys: TablePartitionSpec, - partitionLocationOverrides: Map[String, String], + customPartitionLocations: Map[String, String], partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], fileFormat: FileFormat, @@ -47,7 +47,7 @@ case class InsertIntoHadoopFsRelationCommand( extends RunnableCommand { println("static partition keys: " + staticPartitionKeys) - println("overrides: " + partitionLocationOverrides) + println("overrides: " + customPartitionLocations) override protected def innerChildren: Seq[LogicalPlan] = query :: Nil @@ -113,7 +113,8 @@ case class InsertIntoHadoopFsRelationCommand( plan = query, fileFormat = fileFormat, committer = committer, - outputPath = qualifiedOutputPath.toString, + outputSpec = FileFormatWriter.OutputSpec( + qualifiedOutputPath.toString, customPartitionLocations), hadoopConf = hadoopConf, partitionColumns = partitionColumns, bucketSpec = bucketSpec, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index e849cafef418..f1c5f9ab5067 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -80,7 +80,7 @@ class FileStreamSink( plan = data.logicalPlan, fileFormat = fileFormat, committer = committer, - outputPath = path, + outputSpec = FileFormatWriter.OutputSpec(path, Map.empty), hadoopConf = hadoopConf, partitionColumns = partitionColumns, bucketSpec = None, From 3c8b43ec75401727a1364391f53c7182dbfecf96 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Fri, 4 Nov 2016 19:56:49 -0700 Subject: [PATCH 03/27] Fri Nov 4 19:56:49 PDT 2016 --- .../internal/io/FileCommitProtocol.scala | 12 ++++++-- .../io/HadoopMapReduceCommitProtocol.scala | 29 +++++++++++++++---- 2 files changed, 34 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala index fb8020585cf8..1c4382fa8308 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -80,10 +80,18 @@ abstract class FileCommitProtocol { * 4. bucket id * 5. source specific file extension, e.g. ".snappy.parquet" * - * The "dir" parameter specifies 2, and "ext" parameter specifies both 4 and 5, and the rest + * The directory parameters specify 2, and "ext" parameter specifies both 4 and 5, and the rest * are left to the commit protocol implementation to decide. + * + * @param relativeDir relative path of the final output directory from the base path, or None + * if the file should be placed in the base path or an absolute path. + * @param absoluteDir absolute path of the final output directory. If specified, the base path + * will be ignored. This may not be specified if relativeDir is. + * @param ext extension of the final output file. */ - def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String + def newTaskTempFile( + taskContext: TaskAttemptContext, + relativeDir: Option[String], absoluteDir: Option[String], ext: String): String /** * Commits a task after the writes succeed. Must be called on the executors when running tasks. diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 66ccb6d43770..a65538849f83 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -42,12 +42,23 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) /** OutputCommitter from Hadoop is not serializable so marking it transient. */ @transient private var committer: OutputCommitter = _ + /** + * Tracks files staged by this task for absolute output paths. Unlike files staged by the + * OutputCommitter, we must manually move these to their final locations on task commit. + * TODO(ekl) it would be nice to provide better atomicity for this type of output. + */ + @transient private var taskAbsPathOutputs: List[String] = Nil + protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { context.getOutputFormatClass.newInstance().getOutputCommitter(context) } override def newTaskTempFile( - taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { + taskContext: TaskAttemptContext, + relativeDir: Option[String], absoluteDir: Option[String], ext: String): String = { + if (absoluteDir.isDefined) { + require(!relativeDir.isDefined, "Cannot specify both abs and relative output dirs.") + } // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, // the file name is fine and won't overflow. @@ -60,10 +71,18 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) case _ => path } - dir.map { d => - new Path(new Path(stagingDir, d), filename).toString - }.getOrElse { - new Path(stagingDir, filename).toString + absoluteDir match { + case Some(d) => + val absOutputPath = new Path(d, filename).toString + taskAbsPathOutputs ::= absOutputPath + absOutputPath + case _ => + relativeDir match { + case Some(d) => + new Path(new Path(stagingDir, d), filename).toString + case _ => + new Path(stagingDir, filename).toString + } } } From fb7ba10979f05d8a6955d7c77ecefea52e017dc9 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 7 Nov 2016 15:11:38 -0800 Subject: [PATCH 04/27] Mon Nov 7 15:11:38 PST 2016 --- .../io/HadoopMapReduceCommitProtocol.scala | 7 +- .../plans/logical/basicLogicalOperators.scala | 6 +- .../datasources/DataSourceStrategy.scala | 12 +-- .../datasources/FileFormatWriter.scala | 18 ++++- .../InsertIntoHadoopFsRelationCommand.scala | 66 ++++++++++------- .../ManifestFileCommitProtocol.scala | 9 ++- .../PartitionProviderCompatibilitySuite.scala | 73 +++++++++++++++++-- 7 files changed, 145 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index a65538849f83..154a0ee4c329 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -19,6 +19,8 @@ package org.apache.spark.internal.io import java.util.Date +import scala.collection.mutable.ArrayBuffer + import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter @@ -47,7 +49,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) * OutputCommitter, we must manually move these to their final locations on task commit. * TODO(ekl) it would be nice to provide better atomicity for this type of output. */ - @transient private var taskAbsPathOutputs: List[String] = Nil + @transient private var taskAbsPathOutputs: ArrayBuffer[String] = _ protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { context.getOutputFormatClass.newInstance().getOutputCommitter(context) @@ -74,7 +76,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) absoluteDir match { case Some(d) => val absOutputPath = new Path(d, filename).toString - taskAbsPathOutputs ::= absOutputPath + taskAbsPathOutputs += absOutputPath absOutputPath case _ => relativeDir match { @@ -115,6 +117,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) override def setupTask(taskContext: TaskAttemptContext): Unit = { committer = setupCommitter(taskContext) committer.setupTask(taskContext) + taskAbsPathOutputs = new ArrayBuffer[String] } override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 574caf039d3d..75baba887924 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -357,9 +357,9 @@ case class BroadcastHint(child: LogicalPlan) extends UnaryNode { case class OverwriteOptions( enabled: Boolean, staticPartitionKeys: CatalogTypes.TablePartitionSpec = Map.empty) { - if (staticPartitionKeys.nonEmpty) { - assert(enabled, "Overwrite must be enabled when specifying specific partitions.") - } +// if (staticPartitionKeys.nonEmpty) { +// assert(enabled, "Overwrite must be enabled when specifying specific partitions.") +// } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index eff2026f7fac..37eb3aaa060e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -183,10 +183,10 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { "Cannot overwrite a path that is also being read from.") } - val partitionLocationOverrides: Map[String, String] = + val customPartitionLocations: Map[TablePartitionSpec, String] = if (t.sparkSession.sessionState.conf.manageFilesourcePartitions && l.catalogTable.get.tracksPartitionsInCatalog) { - getPartitionLocationOverrides( + getCustomPartitionLocations( t.sparkSession, l.catalogTable.get, outputPath, overwrite.staticPartitionKeys) } else { Map.empty @@ -212,7 +212,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { val insertCmd = InsertIntoHadoopFsRelationCommand( outputPath, if (overwrite.enabled) overwrite.staticPartitionKeys else Map.empty, - partitionLocationOverrides, + customPartitionLocations, partitionSchema, t.bucketSpec, t.fileFormat, @@ -224,11 +224,11 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { insertCmd } - private def getPartitionLocationOverrides( + private def getCustomPartitionLocations( spark: SparkSession, table: CatalogTable, basePath: Path, - staticPartitionKeys: TablePartitionSpec): Map[String, String] = { + staticPartitionKeys: TablePartitionSpec): Map[TablePartitionSpec, String] = { val hadoopConf = spark.sessionState.newHadoopConf val fs = basePath.getFileSystem(hadoopConf) val qualifiedBasePath = basePath.makeQualified(fs.getUri, fs.getWorkingDirectory) @@ -240,7 +240,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { val catalogLocation = new Path(p.storage.locationUri.get).makeQualified( fs.getUri, fs.getWorkingDirectory).toString if (catalogLocation != defaultLocation) { - Some(defaultLocation -> catalogLocation) + Some(p.spec -> catalogLocation) } else { None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index ecf07a8cb0fe..e5a9ea8d9fb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -48,7 +48,8 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter object FileFormatWriter extends Logging { /** Describes how output files should be placed in the filesystem. */ - case class OutputSpec(outputPath: String, customPartitionLocations: Map[String, String]) + case class OutputSpec( + outputPath: String, customPartitionLocations: Map[TablePartitionSpec, String]) /** A shared job description for all the write tasks. */ private class WriteJobDescription( @@ -60,7 +61,7 @@ object FileFormatWriter extends Logging { val nonPartitionColumns: Seq[Attribute], val bucketSpec: Option[BucketSpec], val path: String, - val customPartitionLocations: Map[String, String]) + val customPartitionLocations: Map[TablePartitionSpec, String]) extends Serializable { assert(AttributeSet(allColumns) == AttributeSet(partitionColumns ++ nonPartitionColumns), @@ -229,6 +230,7 @@ object FileFormatWriter extends Logging { val tmpFilePath = committer.newTaskTempFile( taskAttemptContext, None, + None, description.outputWriterFactory.getFileExtension(taskAttemptContext)) val outputWriter = description.outputWriterFactory.newInstance( @@ -313,7 +315,17 @@ object FileFormatWriter extends Logging { } val ext = bucketId + description.outputWriterFactory.getFileExtension(taskAttemptContext) - val path = committer.newTaskTempFile(taskAttemptContext, partDir, ext) + val customPath = partDir match { + case Some(dir) => + description.customPartitionLocations.get(PartitioningUtils.parsePathFragment(dir)) + case _ => + None + } + val path = if (customPath.isDefined) { + committer.newTaskTempFile(taskAttemptContext, None, customPath, ext) + } else { + committer.newTaskTempFile(taskAttemptContext, partDir, None, ext) + } val newWriter = description.outputWriterFactory.newInstance( path = path, dataSchema = description.nonPartitionColumns.toStructType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index a3d2cbafd0b0..0c8d1e0acd0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import java.io.IOException -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql._ @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution.command.RunnableCommand case class InsertIntoHadoopFsRelationCommand( outputPath: Path, staticPartitionKeys: TablePartitionSpec, - customPartitionLocations: Map[String, String], + customPartitionLocations: Map[TablePartitionSpec, String], partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], fileFormat: FileFormat, @@ -46,9 +46,6 @@ case class InsertIntoHadoopFsRelationCommand( mode: SaveMode) extends RunnableCommand { - println("static partition keys: " + staticPartitionKeys) - println("overrides: " + customPartitionLocations) - override protected def innerChildren: Seq[LogicalPlan] = query :: Nil override def run(sparkSession: SparkSession): Seq[Row] = { @@ -70,26 +67,7 @@ case class InsertIntoHadoopFsRelationCommand( case (SaveMode.ErrorIfExists, true) => throw new AnalysisException(s"path $qualifiedOutputPath already exists.") case (SaveMode.Overwrite, true) => - val suffix = if (staticPartitionKeys.nonEmpty) { - "/" + partitionColumns.flatMap { p => - staticPartitionKeys.get(p.name) match { - case Some(value) => - Some( - PartitioningUtils.escapePathName(p.name) + "=" + - PartitioningUtils.escapePathName(value)) - case None => - None - } - }.mkString("/") - } else { - "" - } - val pathToDelete = qualifiedOutputPath.suffix(suffix) - println("path to delete: " + pathToDelete) - if (fs.exists(pathToDelete) && !fs.delete(pathToDelete, true /* recursively */)) { - throw new IOException(s"Unable to clear output " + - s"directory $qualifiedOutputPath prior to writing to it") - } + deleteMatchingPartitions(fs, qualifiedOutputPath) true case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => true @@ -126,4 +104,42 @@ case class InsertIntoHadoopFsRelationCommand( Seq.empty[Row] } + + /** + * Deletes all partitions that match the specified static prefix. Partitions with custom + * locations are also cleared based on the custom locations map given to this class. + */ + private def deleteMatchingPartitions(fs: FileSystem, qualifiedOutputPath: Path): Unit = { + val staticPartitionPrefix = if (staticPartitionKeys.nonEmpty) { + "/" + partitionColumns.flatMap { p => + staticPartitionKeys.get(p.name) match { + case Some(value) => + Some( + PartitioningUtils.escapePathName(p.name) + "=" + + PartitioningUtils.escapePathName(value)) + case None => + None + } + }.mkString("/") + } else { + "" + } + // first clear the path determined by the static partition keys (e.g. /table/foo=1) + val staticPrefixPath = qualifiedOutputPath.suffix(staticPartitionPrefix) + if (fs.exists(staticPrefixPath) && !fs.delete(staticPrefixPath, true /* recursively */)) { + throw new IOException(s"Unable to clear output " + + s"directory $staticPrefixPath prior to writing to it") + } + // now clear all custom partition locations (e.g. /custom/dir/where/foo=2/bar=4) + for ((spec, customLoc) <- customPartitionLocations) { + assert( + (staticPartitionKeys.toSet -- spec).isEmpty, + "Custom partition location did not match static partitioning keys") + val path = new Path(customLoc) + if (fs.exists(path) && !fs.delete(path, true)) { + throw new IOException(s"Unable to clear partition " + + s"directory $path prior to writing to it") + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index 1fe13fa1623f..03c762813b90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -78,7 +78,12 @@ class ManifestFileCommitProtocol(jobId: String, path: String) } override def newTaskTempFile( - taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { + taskContext: TaskAttemptContext, + relativeDir: Option[String], absoluteDir: Option[String], ext: String): String = { + if (absoluteDir.isDefined) { + throw new UnsupportedOperationException( + "absolute output path not supported by manifest commit protocol") + } // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, // the file name is fine and won't overflow. @@ -86,7 +91,7 @@ class ManifestFileCommitProtocol(jobId: String, path: String) val uuid = UUID.randomUUID.toString val filename = f"part-$split%05d-$uuid$ext" - val file = dir.map { d => + val file = relativeDir.map { d => new Path(new Path(path, d), filename).toString }.getOrElse { new Path(path, filename).toString diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index 14ecfab291ab..e6110d4e37f5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -60,7 +60,6 @@ class PartitionProviderCompatibilitySuite } } -/* test("convert partition provider to hive with repair table") { withTable("test") { withTempDir { dir => @@ -136,7 +135,7 @@ class PartitionProviderCompatibilitySuite } } - test("insert overwrite partition of legacy datasource table overwrites entire table") { + test("insert overwrite partition of legacy datasource table") { withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { withTable("test") { withTempDir { dir => @@ -145,9 +144,9 @@ class PartitionProviderCompatibilitySuite """insert overwrite table test |partition (partCol=1) |select * from range(100)""".stripMargin) - assert(spark.sql("select * from test").count() == 100) + assert(spark.sql("select * from test").count() == 104) - // Dynamic partitions case + // Overwriting entire table spark.sql("insert overwrite table test select id, id from range(10)".stripMargin) assert(spark.sql("select * from test").count() == 10) } @@ -155,7 +154,6 @@ class PartitionProviderCompatibilitySuite } } -*/ test("insert overwrite partition of new datasource table overwrites just partition") { withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { withTable("test") { @@ -188,4 +186,69 @@ class PartitionProviderCompatibilitySuite } } } + + test("insert into and overwrite new datasource tables with partial specs and custom locs") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + withTable("test") { + withTempDir { dir => + spark.range(5).selectExpr("id", "id as p1", "id as p2").write + .partitionBy("p1", "p2") + .mode("overwrite") + .parquet(dir.getAbsolutePath) + spark.sql(s""" + |create table test (id long, p1 int, p2 int) + |using parquet + |options (path "${dir.getAbsolutePath}") + |partitioned by (p1, p2)""".stripMargin) + spark.sql("msck repair table test") + assert(spark.sql("select * from test").count() == 5) + + // dynamic append with partial spec, existing dir + spark.sql("insert into test partition (p1=1, p2) select id, id from range(10)") + assert(spark.sql("select * from test where p1=1").count() == 11) + assert(spark.sql("select * from test where p1=1 and p2=1").count() == 2) + + // dynamic append with full spec, existing dir + spark.sql("insert into test partition (p1=1, p2=1) select id from range(10)") + assert(spark.sql("select * from test where p1=1").count() == 21) + assert(spark.sql("select * from test where p1=1 and p2=1").count() == 12) + + // dynamic append with partial spec, new dir + spark.sql("insert into test partition (p1=100, p2) select id, id from range(10)") + assert(spark.sql("select * from test where p1=100").count() == 10) + + // dynamic append with full spec, new dir + spark.sql("insert into test partition (p1=100, p2=100) select id from range(10)") + assert(spark.sql("select * from test where p1=100").count() == 20) + + // dynamic overwrite with partial spec, existing dir + spark.sql( + "insert overwrite table test partition (p1=1, p2) select id, id from range(100)") + assert(spark.sql("select * from test where p1=1").count() == 100) + + // dynamic overwrite with full spec, existing dir + spark.sql( + "insert overwrite table test partition (p1=1, p2=1) select id from range(100)") + assert(spark.sql("select * from test where p1=1").count() == 199) + assert(spark.sql("select * from test where p1=1 and p2=1").count() == 100) + + // dynamic overwrite with partial spec, new dir + spark.sql( + "insert overwrite table test partition (p1=500, p2) select id, id from range(10)") + assert(spark.sql("select * from test where p1=500").count() == 10) + + // dynamic overwrite with full spec, new dir + spark.sql( + "insert overwrite table test partition (p1=500, p2=500) select id from range(10)") + assert(spark.sql("select * from test where p1=500 and p2=500").count() == 10) + + // overwrite entire table + assert(spark.sql("select * from test").count() == 243) + spark.sql("insert overwrite table test select id, 1, 1 from range(10)") + assert(spark.sql("select * from test").count() == 10) + spark.sql("show partitions test").show() + } + } + } + } } From 33189708d2b40befd93febc13fc33b3f833688ef Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 7 Nov 2016 15:22:33 -0800 Subject: [PATCH 05/27] Mon Nov 7 15:22:33 PST 2016 --- .../datasources/InsertIntoHadoopFsRelationCommand.scala | 3 ++- .../spark/sql/hive/PartitionProviderCompatibilitySuite.scala | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 0c8d1e0acd0b..726ab2966715 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -106,9 +106,10 @@ case class InsertIntoHadoopFsRelationCommand( } /** - * Deletes all partitions that match the specified static prefix. Partitions with custom + * Deletes all partition files that match the specified static prefix. Partitions with custom * locations are also cleared based on the custom locations map given to this class. */ + // TODO(ekl) also delete partition metadata from the catalog private def deleteMatchingPartitions(fs: FileSystem, qualifiedOutputPath: Path): Unit = { val staticPartitionPrefix = if (staticPartitionKeys.nonEmpty) { "/" + partitionColumns.flatMap { p => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index e6110d4e37f5..0f145f74de87 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -246,7 +246,6 @@ class PartitionProviderCompatibilitySuite assert(spark.sql("select * from test").count() == 243) spark.sql("insert overwrite table test select id, 1, 1 from range(10)") assert(spark.sql("select * from test").count() == 10) - spark.sql("show partitions test").show() } } } From aa2536fd954910c418681cda1eb0c6164228e4ec Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 7 Nov 2016 23:44:55 +0800 Subject: [PATCH 06/27] correct several partition related behaviours of ExternalCatalog --- .../catalog/ExternalCatalogUtils.scala | 92 ++++++ .../catalyst/catalog/InMemoryCatalog.scala | 92 +++--- .../sql/catalyst/catalog/interface.scala | 4 + .../catalog/ExternalCatalogSuite.scala | 173 ++++++++--- .../spark/sql/execution/command/ddl.scala | 2 +- .../spark/sql/execution/command/tables.scala | 3 +- .../datasources/FileFormatWriter.scala | 6 +- .../PartitioningAwareFileIndex.scala | 2 - .../datasources/PartitioningUtils.scala | 64 +--- .../sql/execution/command/DDLSuite.scala | 8 +- .../ParquetPartitionDiscoverySuite.scala | 21 +- .../spark/sql/hive/HiveExternalCatalog.scala | 276 +++++++++++------- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 4 +- .../spark/sql/hive/MultiDatabaseSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- 16 files changed, 474 insertions(+), 279 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala new file mode 100644 index 000000000000..5d17189fc852 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -0,0 +1,92 @@ +/* + * 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.catalyst.catalog + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.util.Shell + +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec + +object ExternalCatalogUtils { + // This duplicates default value of Hive `ConfVars.DEFAULTPARTITIONNAME`, since catalyst doesn't + // depend on Hive. + val DEFAULT_PARTITION_NAME = "__HIVE_DEFAULT_PARTITION__" + + ////////////////////////////////////////////////////////////////////////////////////////////////// + // The following string escaping code is mainly copied from Hive (o.a.h.h.common.FileUtils). + ////////////////////////////////////////////////////////////////////////////////////////////////// + + val charToEscape = { + val bitSet = new java.util.BitSet(128) + + /** + * ASCII 01-1F are HTTP control characters that need to be escaped. + * \u000A and \u000D are \n and \r, respectively. + */ + val clist = Array( + '\u0001', '\u0002', '\u0003', '\u0004', '\u0005', '\u0006', '\u0007', '\u0008', '\u0009', + '\n', '\u000B', '\u000C', '\r', '\u000E', '\u000F', '\u0010', '\u0011', '\u0012', '\u0013', + '\u0014', '\u0015', '\u0016', '\u0017', '\u0018', '\u0019', '\u001A', '\u001B', '\u001C', + '\u001D', '\u001E', '\u001F', '"', '#', '%', '\'', '*', '/', ':', '=', '?', '\\', '\u007F', + '{', '[', ']', '^') + + clist.foreach(bitSet.set(_)) + + if (Shell.WINDOWS) { + Array(' ', '<', '>', '|').foreach(bitSet.set(_)) + } + + bitSet + } + + def needsEscaping(c: Char): Boolean = { + c >= 0 && c < charToEscape.size() && charToEscape.get(c) + } + + def escapePathName(path: String): String = { + val builder = new StringBuilder() + path.foreach { c => + if (needsEscaping(c)) { + builder.append('%') + builder.append(f"${c.asInstanceOf[Int]}%02X") + } else { + builder.append(c) + } + } + + builder.toString() + } + + def generatePartitionPath( + spec: TablePartitionSpec, + partitionColumnNames: Seq[String], + tablePath: Path): Path = { + val partitionPathStrings = partitionColumnNames.map { col => + val partitionValue = spec(col) + val partitionString = if (partitionValue == null) { + DEFAULT_PARTITION_NAME + } else { + escapePathName(partitionValue) + } + escapePathName(col) + "=" + partitionString + } + partitionPathStrings.foldLeft(tablePath) { (totalPath, nextPartPath) => + new Path(totalPath, nextPartPath) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 20db81e6f906..dc973ce5f804 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -231,7 +231,7 @@ class InMemoryCatalog( assert(tableMeta.storage.locationUri.isDefined, "Managed table should always have table location, as we will assign a default location " + "to it if it doesn't have one.") - val dir = new Path(tableMeta.storage.locationUri.get) + val dir = new Path(tableMeta.location) try { val fs = dir.getFileSystem(hadoopConfig) fs.delete(dir, true) @@ -259,7 +259,7 @@ class InMemoryCatalog( assert(oldDesc.table.storage.locationUri.isDefined, "Managed table should always have table location, as we will assign a default location " + "to it if it doesn't have one.") - val oldDir = new Path(oldDesc.table.storage.locationUri.get) + val oldDir = new Path(oldDesc.table.location) val newDir = new Path(catalog(db).db.locationUri, newName) try { val fs = oldDir.getFileSystem(hadoopConfig) @@ -355,25 +355,28 @@ class InMemoryCatalog( } } - val tableDir = new Path(catalog(db).db.locationUri, table) - val partitionColumnNames = getTable(db, table).partitionColumnNames + val tableMeta = getTable(db, table) + val partitionColumnNames = tableMeta.partitionColumnNames + val tablePath = new Path(tableMeta.location) // TODO: we should follow hive to roll back if one partition path failed to create. parts.foreach { p => - // If location is set, the partition is using an external partition location and we don't - // need to handle its directory. - if (p.storage.locationUri.isEmpty) { - val partitionPath = partitionColumnNames.flatMap { col => - p.spec.get(col).map(col + "=" + _) - }.mkString("/") - try { - val fs = tableDir.getFileSystem(hadoopConfig) - fs.mkdirs(new Path(tableDir, partitionPath)) - } catch { - case e: IOException => - throw new SparkException(s"Unable to create partition path $partitionPath", e) + val partitionPath = p.storage.locationUri.map(new Path(_)).getOrElse { + ExternalCatalogUtils.generatePartitionPath(p.spec, partitionColumnNames, tablePath) + } + + try { + val fs = tablePath.getFileSystem(hadoopConfig) + if (!fs.exists(partitionPath)) { + fs.mkdirs(partitionPath) } + } catch { + case e: IOException => + throw new SparkException(s"Unable to create partition path $partitionPath", e) } - existingParts.put(p.spec, p) + + existingParts.put( + p.spec, + p.copy(storage = p.storage.copy(locationUri = Some(partitionPath.toString)))) } } @@ -392,19 +395,15 @@ class InMemoryCatalog( } } - val tableDir = new Path(catalog(db).db.locationUri, table) - val partitionColumnNames = getTable(db, table).partitionColumnNames - // TODO: we should follow hive to roll back if one partition path failed to delete. + val shouldRemovePartitionLocation = getTable(db, table).tableType == CatalogTableType.MANAGED + // TODO: we should follow hive to roll back if one partition path failed to delete, and support + // partial partition spec. partSpecs.foreach { p => - // If location is set, the partition is using an external partition location and we don't - // need to handle its directory. - if (existingParts.contains(p) && existingParts(p).storage.locationUri.isEmpty) { - val partitionPath = partitionColumnNames.flatMap { col => - p.get(col).map(col + "=" + _) - }.mkString("/") + if (existingParts.contains(p) && shouldRemovePartitionLocation) { + val partitionPath = new Path(existingParts(p).storage.locationUri.get) try { - val fs = tableDir.getFileSystem(hadoopConfig) - fs.delete(new Path(tableDir, partitionPath), true) + val fs = partitionPath.getFileSystem(hadoopConfig) + fs.delete(partitionPath, true) } catch { case e: IOException => throw new SparkException(s"Unable to delete partition path $partitionPath", e) @@ -423,33 +422,34 @@ class InMemoryCatalog( requirePartitionsExist(db, table, specs) requirePartitionsNotExist(db, table, newSpecs) - val tableDir = new Path(catalog(db).db.locationUri, table) - val partitionColumnNames = getTable(db, table).partitionColumnNames + val tableMeta = getTable(db, table) + val partitionColumnNames = tableMeta.partitionColumnNames + val tablePath = new Path(tableMeta.location) + val shouldUpdatePartitionLocation = getTable(db, table).tableType == CatalogTableType.MANAGED + val existingParts = catalog(db).tables(table).partitions // TODO: we should follow hive to roll back if one partition path failed to rename. specs.zip(newSpecs).foreach { case (oldSpec, newSpec) => - val newPart = getPartition(db, table, oldSpec).copy(spec = newSpec) - val existingParts = catalog(db).tables(table).partitions - - // If location is set, the partition is using an external partition location and we don't - // need to handle its directory. - if (newPart.storage.locationUri.isEmpty) { - val oldPath = partitionColumnNames.flatMap { col => - oldSpec.get(col).map(col + "=" + _) - }.mkString("/") - val newPath = partitionColumnNames.flatMap { col => - newSpec.get(col).map(col + "=" + _) - }.mkString("/") + val oldPartition = getPartition(db, table, oldSpec) + val newPartition = if (shouldUpdatePartitionLocation) { + val oldPartPath = new Path(oldPartition.storage.locationUri.get) + val newPartPath = ExternalCatalogUtils.generatePartitionPath( + newSpec, partitionColumnNames, tablePath) try { - val fs = tableDir.getFileSystem(hadoopConfig) - fs.rename(new Path(tableDir, oldPath), new Path(tableDir, newPath)) + val fs = tablePath.getFileSystem(hadoopConfig) + fs.rename(oldPartPath, newPartPath) } catch { case e: IOException => - throw new SparkException(s"Unable to rename partition path $oldPath", e) + throw new SparkException(s"Unable to rename partition path $oldPartPath", e) } + oldPartition.copy( + spec = newSpec, + storage = oldPartition.storage.copy(locationUri = Some(newPartPath.toString))) + } else { + oldPartition.copy(spec = newSpec) } existingParts.remove(oldSpec) - existingParts.put(newSpec, newPart) + existingParts.put(newSpec, newPartition) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 34748a04859a..b3b84eaadbb7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -171,6 +171,10 @@ case class CatalogTable( throw new AnalysisException(s"table $identifier did not specify database") } + def location: String = storage.locationUri.getOrElse { + throw new AnalysisException(s"table $identifier did not specify locationUri") + } + /** Return the fully qualified name of this table, assuming the database was specified. */ def qualifiedName: String = identifier.unquotedString diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 34bdfc8a9871..bc9bcccdb961 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.catalog import java.io.File import java.net.URI +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite @@ -173,6 +175,23 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac } } + test("create managed table without table location") { + val catalog = newBasicCatalog() + val table = CatalogTable( + identifier = TableIdentifier("myTable", Some("db1")), + tableType = CatalogTableType.MANAGED, + storage = storageFormat, + schema = new StructType() + .add("col1", "int") + .add("col2", "string"), + provider = Some("hive")) + catalog.createTable(table, ignoreIfExists = false) + + val tableLocation = new Path(catalog.getTable("db1", "myTable").location) + val defaultTableLocation = new Path(catalog.getDatabase("db1").locationUri, "myTable") + assert(tableLocation == defaultTableLocation) + } + test("drop table") { val catalog = newBasicCatalog() assert(catalog.listTables("db2").toSet == Set("tbl1", "tbl2")) @@ -320,6 +339,33 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac catalog.createPartitions("db2", "tbl2", Seq(part1), ignoreIfExists = true) } + test("create partitions without location") { + val catalog = newBasicCatalog() + val table = CatalogTable( + identifier = TableIdentifier("tbl", Some("db1")), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat(None, None, None, None, false, Map.empty), + schema = new StructType() + .add("col1", "int") + .add("col2", "string") + .add("partCol1", "int") + .add("partCol2", "string"), + provider = Some("hive"), + partitionColumnNames = Seq("partCol1", "partCol2")) + catalog.createTable(table, ignoreIfExists = false) + + val partition = CatalogTablePartition(Map("partCol1" -> "1", "partCol2" -> "2"), storageFormat) + catalog.createPartitions("db1", "tbl", Seq(partition), ignoreIfExists = false) + + val partitionLocation = catalog.getPartition( + "db1", + "tbl", + Map("partCol1" -> "1", "partCol2" -> "2")).storage.locationUri.get + val tableLocation = catalog.getTable("db1", "tbl").location + val defaultPartitionLocation = new Path(new Path(tableLocation, "partCol1=1"), "partCol2=2") + assert(new Path(partitionLocation) == defaultPartitionLocation) + } + test("list partitions with partial partition spec") { val catalog = newBasicCatalog() val parts = catalog.listPartitions("db2", "tbl2", Some(Map("a" -> "1"))) @@ -399,6 +445,46 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac intercept[AnalysisException] { catalog.getPartition("db2", "tbl2", part2.spec) } } + test("rename partitions should update the location for managed table") { + val catalog = newBasicCatalog() + val table = CatalogTable( + identifier = TableIdentifier("tbl", Some("db1")), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat(None, None, None, None, false, Map.empty), + schema = new StructType() + .add("col1", "int") + .add("col2", "string") + .add("partCol1", "int") + .add("partCol2", "string"), + provider = Some("hive"), + partitionColumnNames = Seq("partCol1", "partCol2")) + catalog.createTable(table, ignoreIfExists = false) + + val tableLocation = catalog.getTable("db1", "tbl").location + + val mixedCasePart1 = CatalogTablePartition( + Map("partCol1" -> "1", "partCol2" -> "2"), storageFormat) + val mixedCasePart2 = CatalogTablePartition( + Map("partCol1" -> "3", "partCol2" -> "4"), storageFormat) + + catalog.createPartitions("db1", "tbl", Seq(mixedCasePart1), ignoreIfExists = false) + assert( + new Path(catalog.getPartition("db1", "tbl", mixedCasePart1.spec).storage.locationUri.get) == + new Path(new Path(tableLocation, "partCol1=1"), "partCol2=2")) + + catalog.renamePartitions("db1", "tbl", Seq(mixedCasePart1.spec), Seq(mixedCasePart2.spec)) + assert( + new Path(catalog.getPartition("db1", "tbl", mixedCasePart2.spec).storage.locationUri.get) == + new Path(new Path(tableLocation, "partCol1=3"), "partCol2=4")) + + // For external tables, RENAME PARTITION should not update the partition location. + val existingPartLoc = catalog.getPartition("db2", "tbl2", part1.spec).storage.locationUri.get + catalog.renamePartitions("db2", "tbl2", Seq(part1.spec), Seq(part3.spec)) + assert( + new Path(catalog.getPartition("db2", "tbl2", part3.spec).storage.locationUri.get) == + new Path(existingPartLoc)) + } + test("rename partitions when database/table does not exist") { val catalog = newBasicCatalog() intercept[AnalysisException] { @@ -419,11 +505,6 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac test("alter partitions") { val catalog = newBasicCatalog() try { - // Note: Before altering table partitions in Hive, you *must* set the current database - // to the one that contains the table of interest. Otherwise you will end up with the - // most helpful error message ever: "Unable to alter partition. alter is not possible." - // See HIVE-2742 for more detail. - catalog.setCurrentDatabase("db2") val newLocation = newUriForDatabase() val newSerde = "com.sparkbricks.text.EasySerde" val newSerdeProps = Map("spark" -> "bricks", "compressed" -> "false") @@ -591,7 +672,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac val catalog = newBasicCatalog() val db = catalog.getDatabase("db1") val table = CatalogTable( - identifier = TableIdentifier("my_table", Some("db1")), + identifier = TableIdentifier("myTable", Some("db1")), tableType = CatalogTableType.MANAGED, storage = CatalogStorageFormat(None, None, None, None, false, Map.empty), schema = new StructType().add("a", "int").add("b", "string"), @@ -599,17 +680,17 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac ) catalog.createTable(table, ignoreIfExists = false) - assert(exists(db.locationUri, "my_table")) + assert(exists(db.locationUri, "myTable")) - catalog.renameTable("db1", "my_table", "your_table") - assert(!exists(db.locationUri, "my_table")) - assert(exists(db.locationUri, "your_table")) + catalog.renameTable("db1", "myTable", "yourTable") + assert(!exists(db.locationUri, "myTable")) + assert(exists(db.locationUri, "yourTable")) - catalog.dropTable("db1", "your_table", ignoreIfNotExists = false, purge = false) - assert(!exists(db.locationUri, "your_table")) + catalog.dropTable("db1", "yourTable", ignoreIfNotExists = false, purge = false) + assert(!exists(db.locationUri, "yourTable")) val externalTable = CatalogTable( - identifier = TableIdentifier("external_table", Some("db1")), + identifier = TableIdentifier("externalTable", Some("db1")), tableType = CatalogTableType.EXTERNAL, storage = CatalogStorageFormat( Some(Utils.createTempDir().getAbsolutePath), @@ -618,12 +699,11 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac provider = Some("hive") ) catalog.createTable(externalTable, ignoreIfExists = false) - assert(!exists(db.locationUri, "external_table")) + assert(!exists(db.locationUri, "externalTable")) } test("create/drop/rename partitions should create/delete/rename the directory") { val catalog = newBasicCatalog() - val databaseDir = catalog.getDatabase("db1").locationUri val table = CatalogTable( identifier = TableIdentifier("tbl", Some("db1")), tableType = CatalogTableType.MANAGED, @@ -631,34 +711,61 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac schema = new StructType() .add("col1", "int") .add("col2", "string") - .add("a", "int") - .add("b", "string"), + .add("partCol1", "int") + .add("partCol2", "string"), provider = Some("hive"), - partitionColumnNames = Seq("a", "b") - ) + partitionColumnNames = Seq("partCol1", "partCol2")) catalog.createTable(table, ignoreIfExists = false) + val tableLocation = catalog.getTable("db1", "tbl").location + + val part1 = CatalogTablePartition(Map("partCol1" -> "1", "partCol2" -> "2"), storageFormat) + val part2 = CatalogTablePartition(Map("partCol1" -> "3", "partCol2" -> "4"), storageFormat) + val part3 = CatalogTablePartition(Map("partCol1" -> "5", "partCol2" -> "6"), storageFormat) + catalog.createPartitions("db1", "tbl", Seq(part1, part2), ignoreIfExists = false) - assert(exists(databaseDir, "tbl", "a=1", "b=2")) - assert(exists(databaseDir, "tbl", "a=3", "b=4")) + assert(exists(tableLocation, "partCol1=1", "partCol2=2")) + assert(exists(tableLocation, "partCol1=3", "partCol2=4")) catalog.renamePartitions("db1", "tbl", Seq(part1.spec), Seq(part3.spec)) - assert(!exists(databaseDir, "tbl", "a=1", "b=2")) - assert(exists(databaseDir, "tbl", "a=5", "b=6")) + assert(!exists(tableLocation, "partCol1=1", "partCol2=2")) + assert(exists(tableLocation, "partCol1=5", "partCol2=6")) catalog.dropPartitions("db1", "tbl", Seq(part2.spec, part3.spec), ignoreIfNotExists = false, purge = false) - assert(!exists(databaseDir, "tbl", "a=3", "b=4")) - assert(!exists(databaseDir, "tbl", "a=5", "b=6")) + assert(!exists(tableLocation, "partCol1=3", "partCol2=4")) + assert(!exists(tableLocation, "partCol1=5", "partCol2=6")) - val externalPartition = CatalogTablePartition( - Map("a" -> "7", "b" -> "8"), + val tempPath = Utils.createTempDir() + // create partition with existing directory is OK. + val partWithExistingDir = CatalogTablePartition( + Map("partCol1" -> "7", "partCol2" -> "8"), CatalogStorageFormat( - Some(Utils.createTempDir().getAbsolutePath), - None, None, None, false, Map.empty) - ) - catalog.createPartitions("db1", "tbl", Seq(externalPartition), ignoreIfExists = false) - assert(!exists(databaseDir, "tbl", "a=7", "b=8")) + Some(tempPath.getAbsolutePath), + None, None, None, false, Map.empty)) + catalog.createPartitions("db1", "tbl", Seq(partWithExistingDir), ignoreIfExists = false) + + tempPath.delete() + // create partition with non-existing directory will create that directory. + val partWithNonExistingDir = CatalogTablePartition( + Map("partCol1" -> "9", "partCol2" -> "10"), + CatalogStorageFormat( + Some(tempPath.getAbsolutePath), + None, None, None, false, Map.empty)) + catalog.createPartitions("db1", "tbl", Seq(partWithNonExistingDir), ignoreIfExists = false) + assert(tempPath.exists()) + } + + test("drop partition from external table should not delete the directory") { + val catalog = newBasicCatalog() + catalog.createPartitions("db2", "tbl1", Seq(part1), ignoreIfExists = false) + + val partPath = new Path(catalog.getPartition("db2", "tbl1", part1.spec).storage.locationUri.get) + val fs = partPath.getFileSystem(new Configuration) + assert(fs.exists(partPath)) + + catalog.dropPartitions("db2", "tbl1", Seq(part1.spec), ignoreIfNotExists = false, purge = false) + assert(fs.exists(partPath)) } } @@ -731,7 +838,7 @@ abstract class CatalogTestUtils { CatalogTable( identifier = TableIdentifier(name, database), tableType = CatalogTableType.EXTERNAL, - storage = storageFormat, + storage = storageFormat.copy(locationUri = Some(Utils.createTempDir().getAbsolutePath)), schema = new StructType() .add("col1", "int") .add("col2", "string") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 8500ab460a1b..c2862b4d73e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -500,7 +500,7 @@ case class AlterTableRecoverPartitionsCommand( s"location provided: $tableIdentWithDB") } - val root = new Path(table.storage.locationUri.get) + val root = new Path(table.location) logInfo(s"Recover all the partitions in $root") val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 3cfa639a2fc1..bccb5ea34e66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -701,7 +701,8 @@ case class ShowPartitionsCommand( private def getPartName(spec: TablePartitionSpec, partColNames: Seq[String]): String = { partColNames.map { name => - PartitioningUtils.escapePathName(name) + "=" + PartitioningUtils.escapePathName(spec(name)) + ExternalCatalogUtils.escapePathName(name) + "=" + + ExternalCatalogUtils.escapePathName(spec(name)) }.mkString(File.separator) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index e404dcd5452b..0f8ed9e23fe3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -32,7 +32,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage import org.apache.spark.sql.{Dataset, SparkSession} -import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, ExternalCatalogUtils} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning @@ -281,11 +281,11 @@ object FileFormatWriter extends Logging { private def partitionStringExpression: Seq[Expression] = { description.partitionColumns.zipWithIndex.flatMap { case (c, i) => val escaped = ScalaUDF( - PartitioningUtils.escapePathName _, + ExternalCatalogUtils.escapePathName _, StringType, Seq(Cast(c, StringType)), Seq(StringType)) - val str = If(IsNull(c), Literal(PartitioningUtils.DEFAULT_PARTITION_NAME), escaped) + val str = If(IsNull(c), Literal(ExternalCatalogUtils.DEFAULT_PARTITION_NAME), escaped) val partitionName = Literal(c.name + "=") :: str :: Nil if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index a8a722dd3c62..3740caa22c37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -128,7 +128,6 @@ abstract class PartitioningAwareFileIndex( case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => val spec = PartitioningUtils.parsePartitions( leafDirs, - PartitioningUtils.DEFAULT_PARTITION_NAME, typeInference = false, basePaths = basePaths) @@ -148,7 +147,6 @@ abstract class PartitioningAwareFileIndex( case _ => PartitioningUtils.parsePartitions( leafDirs, - PartitioningUtils.DEFAULT_PARTITION_NAME, typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, basePaths = basePaths) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index b51b41869bf0..abb5ec558474 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -25,12 +25,12 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Try import org.apache.hadoop.fs.Path -import org.apache.hadoop.util.Shell import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.types._ @@ -56,9 +56,6 @@ object PartitionSpec { } object PartitioningUtils { - // This duplicates default value of Hive `ConfVars.DEFAULTPARTITIONNAME`, since sql/core doesn't - // depend on Hive. - val DEFAULT_PARTITION_NAME = "__HIVE_DEFAULT_PARTITION__" private[datasources] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { @@ -90,12 +87,11 @@ object PartitioningUtils { */ private[datasources] def parsePartitions( paths: Seq[Path], - defaultPartitionName: String, typeInference: Boolean, basePaths: Set[Path]): PartitionSpec = { // First, we need to parse every partition's path and see if we can find partition values. val (partitionValues, optDiscoveredBasePaths) = paths.map { path => - parsePartition(path, defaultPartitionName, typeInference, basePaths) + parsePartition(path, typeInference, basePaths) }.unzip // We create pairs of (path -> path's partition value) here @@ -173,7 +169,6 @@ object PartitioningUtils { */ private[datasources] def parsePartition( path: Path, - defaultPartitionName: String, typeInference: Boolean, basePaths: Set[Path]): (Option[PartitionValues], Option[Path]) = { val columns = ArrayBuffer.empty[(String, Literal)] @@ -196,7 +191,7 @@ object PartitioningUtils { // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1. // Once we get the string, we try to parse it and find the partition column and value. val maybeColumn = - parsePartitionColumn(currentPath.getName, defaultPartitionName, typeInference) + parsePartitionColumn(currentPath.getName, typeInference) maybeColumn.foreach(columns += _) // Now, we determine if we should stop. @@ -228,7 +223,6 @@ object PartitioningUtils { private def parsePartitionColumn( columnSpec: String, - defaultPartitionName: String, typeInference: Boolean): Option[(String, Literal)] = { val equalSignIndex = columnSpec.indexOf('=') if (equalSignIndex == -1) { @@ -240,7 +234,7 @@ object PartitioningUtils { val rawColumnValue = columnSpec.drop(equalSignIndex + 1) assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - val literal = inferPartitionColumnValue(rawColumnValue, defaultPartitionName, typeInference) + val literal = inferPartitionColumnValue(rawColumnValue, typeInference) Some(columnName -> literal) } } @@ -355,7 +349,6 @@ object PartitioningUtils { */ private[datasources] def inferPartitionColumnValue( raw: String, - defaultPartitionName: String, typeInference: Boolean): Literal = { val decimalTry = Try { // `BigDecimal` conversion can fail when the `field` is not a form of number. @@ -380,14 +373,14 @@ object PartitioningUtils { .orElse(Try(Literal(JTimestamp.valueOf(unescapePathName(raw))))) // Then falls back to string .getOrElse { - if (raw == defaultPartitionName) { + if (raw == ExternalCatalogUtils.DEFAULT_PARTITION_NAME) { Literal.create(null, NullType) } else { Literal.create(unescapePathName(raw), StringType) } } } else { - if (raw == defaultPartitionName) { + if (raw == ExternalCatalogUtils.DEFAULT_PARTITION_NAME) { Literal.create(null, NullType) } else { Literal.create(unescapePathName(raw), StringType) @@ -451,51 +444,6 @@ object PartitioningUtils { } } - ////////////////////////////////////////////////////////////////////////////////////////////////// - // The following string escaping code is mainly copied from Hive (o.a.h.h.common.FileUtils). - ////////////////////////////////////////////////////////////////////////////////////////////////// - - val charToEscape = { - val bitSet = new java.util.BitSet(128) - - /** - * ASCII 01-1F are HTTP control characters that need to be escaped. - * \u000A and \u000D are \n and \r, respectively. - */ - val clist = Array( - '\u0001', '\u0002', '\u0003', '\u0004', '\u0005', '\u0006', '\u0007', '\u0008', '\u0009', - '\n', '\u000B', '\u000C', '\r', '\u000E', '\u000F', '\u0010', '\u0011', '\u0012', '\u0013', - '\u0014', '\u0015', '\u0016', '\u0017', '\u0018', '\u0019', '\u001A', '\u001B', '\u001C', - '\u001D', '\u001E', '\u001F', '"', '#', '%', '\'', '*', '/', ':', '=', '?', '\\', '\u007F', - '{', '[', ']', '^') - - clist.foreach(bitSet.set(_)) - - if (Shell.WINDOWS) { - Array(' ', '<', '>', '|').foreach(bitSet.set(_)) - } - - bitSet - } - - def needsEscaping(c: Char): Boolean = { - c >= 0 && c < charToEscape.size() && charToEscape.get(c) - } - - def escapePathName(path: String): String = { - val builder = new StringBuilder() - path.foreach { c => - if (needsEscaping(c)) { - builder.append('%') - builder.append(f"${c.asInstanceOf[Int]}%02X") - } else { - builder.append(c) - } - } - - builder.toString() - } - def unescapePathName(path: String): String = { val sb = new StringBuilder var i = 0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 864af8d578b1..b663ee0c3eeb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -875,7 +875,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1)) val part2 = Map("a" -> "2", "b" -> "6") - val root = new Path(catalog.getTableMetadata(tableIdent).storage.locationUri.get) + val root = new Path(catalog.getTableMetadata(tableIdent).location) val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) // valid fs.mkdirs(new Path(new Path(root, "a=1"), "b=5")) @@ -1133,7 +1133,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } assert(catalog.getTableMetadata(tableIdent).storage.locationUri.isDefined) assert(catalog.getTableMetadata(tableIdent).storage.properties.isEmpty) - assert(catalog.getPartition(tableIdent, partSpec).storage.locationUri.isEmpty) + assert(catalog.getPartition(tableIdent, partSpec).storage.locationUri.isDefined) assert(catalog.getPartition(tableIdent, partSpec).storage.properties.isEmpty) // Verify that the location is set to the expected string def verifyLocation(expected: String, spec: Option[TablePartitionSpec] = None): Unit = { @@ -1296,9 +1296,9 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql("ALTER TABLE dbx.tab1 ADD IF NOT EXISTS " + "PARTITION (a='2', b='6') LOCATION 'paris' PARTITION (a='3', b='7')") assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == Set(part1, part2, part3)) - assert(catalog.getPartition(tableIdent, part1).storage.locationUri.isEmpty) + assert(catalog.getPartition(tableIdent, part1).storage.locationUri.isDefined) assert(catalog.getPartition(tableIdent, part2).storage.locationUri == Option("paris")) - assert(catalog.getPartition(tableIdent, part3).storage.locationUri.isEmpty) + assert(catalog.getPartition(tableIdent, part3).storage.locationUri.isDefined) // add partitions without explicitly specifying database catalog.setCurrentDatabase("dbx") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 120a3a2ef33a..22e35a1bc0b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -29,6 +29,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition} @@ -48,11 +49,11 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha import PartitioningUtils._ import testImplicits._ - val defaultPartitionName = "__HIVE_DEFAULT_PARTITION__" + val defaultPartitionName = ExternalCatalogUtils.DEFAULT_PARTITION_NAME test("column type inference") { def check(raw: String, literal: Literal): Unit = { - assert(inferPartitionColumnValue(raw, defaultPartitionName, true) === literal) + assert(inferPartitionColumnValue(raw, true) === literal) } check("10", Literal.create(10, IntegerType)) @@ -76,7 +77,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/path/a=10.5/b=hello") var exception = intercept[AssertionError] { - parsePartitions(paths.map(new Path(_)), defaultPartitionName, true, Set.empty[Path]) + parsePartitions(paths.map(new Path(_)), true, Set.empty[Path]) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -88,7 +89,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha parsePartitions( paths.map(new Path(_)), - defaultPartitionName, true, Set(new Path("hdfs://host:9000/path/"))) @@ -101,7 +101,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha parsePartitions( paths.map(new Path(_)), - defaultPartitionName, true, Set(new Path("hdfs://host:9000/path/something=true/table"))) @@ -114,7 +113,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha parsePartitions( paths.map(new Path(_)), - defaultPartitionName, true, Set(new Path("hdfs://host:9000/path/table=true"))) @@ -127,7 +125,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha exception = intercept[AssertionError] { parsePartitions( paths.map(new Path(_)), - defaultPartitionName, true, Set(new Path("hdfs://host:9000/path/"))) } @@ -147,7 +144,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha exception = intercept[AssertionError] { parsePartitions( paths.map(new Path(_)), - defaultPartitionName, true, Set(new Path("hdfs://host:9000/tmp/tables/"))) } @@ -156,13 +152,13 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partition") { def check(path: String, expected: Option[PartitionValues]): Unit = { - val actual = parsePartition(new Path(path), defaultPartitionName, true, Set.empty[Path])._1 + val actual = parsePartition(new Path(path), true, Set.empty[Path])._1 assert(expected === actual) } def checkThrows[T <: Throwable: Manifest](path: String, expected: String): Unit = { val message = intercept[T] { - parsePartition(new Path(path), defaultPartitionName, true, Set.empty[Path]) + parsePartition(new Path(path), true, Set.empty[Path]) }.getMessage assert(message.contains(expected)) @@ -204,7 +200,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha // when the basePaths is the same as the path to a leaf directory val partitionSpec1: Option[PartitionValues] = parsePartition( path = new Path("file://path/a=10"), - defaultPartitionName = defaultPartitionName, typeInference = true, basePaths = Set(new Path("file://path/a=10")))._1 @@ -213,7 +208,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha // when the basePaths is the path to a base directory of leaf directories val partitionSpec2: Option[PartitionValues] = parsePartition( path = new Path("file://path/a=10"), - defaultPartitionName = defaultPartitionName, typeInference = true, basePaths = Set(new Path("file://path")))._1 @@ -231,7 +225,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha val actualSpec = parsePartitions( paths.map(new Path(_)), - defaultPartitionName, true, rootPaths) assert(actualSpec === spec) @@ -314,7 +307,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partitions with type inference disabled") { def check(paths: Seq[String], spec: PartitionSpec): Unit = { val actualSpec = - parsePartitions(paths.map(new Path(_)), defaultPartitionName, false, Set.empty[Path]) + parsePartitions(paths.map(new Path(_)), false, Set.empty[Path]) assert(actualSpec === spec) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index b537061d0d22..9d18c104c651 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -196,25 +196,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat if (tableDefinition.tableType == VIEW) { client.createTable(tableDefinition, ignoreIfExists) - } else if (tableDefinition.provider.get == DDLUtils.HIVE_PROVIDER) { - // Here we follow data source tables and put table metadata like provider, schema, etc. in - // table properties, so that we can work around the Hive metastore issue about not case - // preserving and make Hive serde table support mixed-case column names. - val tableWithDataSourceProps = tableDefinition.copy( - properties = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition)) - client.createTable(tableWithDataSourceProps, ignoreIfExists) } else { - // To work around some hive metastore issues, e.g. not case-preserving, bad decimal type - // support, no column nullability, etc., we should do some extra works before saving table - // metadata into Hive metastore: - // 1. Put table metadata like provider, schema, etc. in table properties. - // 2. Check if this table is hive compatible. - // 2.1 If it's not hive compatible, set location URI, schema, partition columns and bucket - // spec to empty and save table metadata to Hive. - // 2.2 If it's hive compatible, set serde information in table metadata and try to save - // it to Hive. If it fails, treat it as not hive compatible and go back to 2.1 - val tableProperties = tableMetaToTableProps(tableDefinition) - // Ideally we should not create a managed table with location, but Hive serde table can // specify location for managed table. And in [[CreateDataSourceTableAsSelectCommand]] we have // to create the table directory and write out data before we create this table, to avoid @@ -226,106 +208,139 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } else { tableDefinition.storage.locationUri } - // Ideally we should also put `locationUri` in table properties like provider, schema, etc. - // However, in older version of Spark we already store table location in storage properties - // with key "path". Here we keep this behaviour for backward compatibility. - val storagePropsWithLocation = tableDefinition.storage.properties ++ - tableLocation.map("path" -> _) - - // converts the table metadata to Spark SQL specific format, i.e. set data schema, names and - // bucket specification to empty. Note that partition columns are retained, so that we can - // call partition-related Hive API later. - def newSparkSQLSpecificMetastoreTable(): CatalogTable = { - tableDefinition.copy( - // Hive only allows directory paths as location URIs while Spark SQL data source tables - // also allow file paths. For non-hive-compatible format, we should not set location URI - // to avoid hive metastore to throw exception. - storage = tableDefinition.storage.copy( - locationUri = None, - properties = storagePropsWithLocation), - schema = tableDefinition.partitionSchema, - bucketSpec = None, - properties = tableDefinition.properties ++ tableProperties) + + if (tableDefinition.provider.get == DDLUtils.HIVE_PROVIDER) { + val tableWithDataSourceProps = tableDefinition.copy( + // We can't leave `locationUri` empty and count on Hive metastore to set a default table + // location, because Hive metastore is not case preserving and the table name is always + // lower cased when appear in the default table path, which is not expected. + storage = tableDefinition.storage.copy(locationUri = tableLocation), + // Here we follow data source tables and put table metadata like provider, schema, etc. in + // table properties, so that we can work around the Hive metastore issue about not case + // preserving and make Hive serde table support mixed-case column names. + properties = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition)) + client.createTable(tableWithDataSourceProps, ignoreIfExists) + } else { + createDataSourceTable( + tableDefinition.withNewStorage(locationUri = tableLocation), + ignoreIfExists) } + } + } - // converts the table metadata to Hive compatible format, i.e. set the serde information. - def newHiveCompatibleMetastoreTable(serde: HiveSerDe): CatalogTable = { - val location = if (tableDefinition.tableType == EXTERNAL) { - // When we hit this branch, we are saving an external data source table with hive - // compatible format, which means the data source is file-based and must have a `path`. - require(tableDefinition.storage.locationUri.isDefined, - "External file-based data source table must have a `path` entry in storage properties.") - Some(new Path(tableDefinition.storage.locationUri.get).toUri.toString) - } else { - None - } + private def createDataSourceTable( + tableDefinition: CatalogTable, + ignoreIfExists: Boolean): Unit = { + // To work around some hive metastore issues, e.g. not case-preserving, bad decimal type + // support, no column nullability, etc., we should do some extra works before saving table + // metadata into Hive metastore: + // 1. Put table metadata like provider, schema, etc. in table properties. + // 2. Check if this table is hive compatible. + // 2.1 If it's not hive compatible, set location URI, schema, partition columns and bucket + // spec to empty and save table metadata to Hive. + // 2.2 If it's hive compatible, set serde information in table metadata and try to save + // it to Hive. If it fails, treat it as not hive compatible and go back to 2.1 + val tableProperties = tableMetaToTableProps(tableDefinition) + + // Ideally we should also put `locationUri` in table properties like provider, schema, etc. + // However, in older version of Spark we already store table location in storage properties + // with key "path". Here we keep this behaviour for backward compatibility. + val storagePropsWithLocation = tableDefinition.storage.properties ++ + tableDefinition.storage.locationUri.map("path" -> _) + + // converts the table metadata to Spark SQL specific format, i.e. set data schema, names and + // bucket specification to empty. Note that partition columns are retained, so that we can + // call partition-related Hive API later. + def newSparkSQLSpecificMetastoreTable(): CatalogTable = { + tableDefinition.copy( + // Hive only allows directory paths as location URIs while Spark SQL data source tables + // also allow file paths. For non-hive-compatible format, we should not set location URI + // to avoid hive metastore to throw exception. + storage = tableDefinition.storage.copy( + locationUri = None, + properties = storagePropsWithLocation), + schema = tableDefinition.partitionSchema, + bucketSpec = None, + properties = tableDefinition.properties ++ tableProperties) + } - tableDefinition.copy( - storage = tableDefinition.storage.copy( - locationUri = location, - inputFormat = serde.inputFormat, - outputFormat = serde.outputFormat, - serde = serde.serde, - properties = storagePropsWithLocation - ), - properties = tableDefinition.properties ++ tableProperties) + // converts the table metadata to Hive compatible format, i.e. set the serde information. + def newHiveCompatibleMetastoreTable(serde: HiveSerDe): CatalogTable = { + val location = if (tableDefinition.tableType == EXTERNAL) { + // When we hit this branch, we are saving an external data source table with hive + // compatible format, which means the data source is file-based and must have a `path`. + require(tableDefinition.storage.locationUri.isDefined, + "External file-based data source table must have a `path` entry in storage properties.") + Some(new Path(tableDefinition.location).toUri.toString) + } else { + None } - val qualifiedTableName = tableDefinition.identifier.quotedString - val maybeSerde = HiveSerDe.sourceToSerDe(tableDefinition.provider.get) - val skipHiveMetadata = tableDefinition.storage.properties - .getOrElse("skipHiveMetadata", "false").toBoolean - - val (hiveCompatibleTable, logMessage) = maybeSerde match { - case _ if skipHiveMetadata => - val message = - s"Persisting data source table $qualifiedTableName into Hive metastore in" + - "Spark SQL specific format, which is NOT compatible with Hive." - (None, message) - - // our bucketing is un-compatible with hive(different hash function) - case _ if tableDefinition.bucketSpec.nonEmpty => - val message = - s"Persisting bucketed data source table $qualifiedTableName into " + - "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " - (None, message) - - case Some(serde) => - val message = - s"Persisting file based data source table $qualifiedTableName into " + - s"Hive metastore in Hive compatible format." - (Some(newHiveCompatibleMetastoreTable(serde)), message) - - case _ => - val provider = tableDefinition.provider.get - val message = - s"Couldn't find corresponding Hive SerDe for data source provider $provider. " + - s"Persisting data source table $qualifiedTableName into Hive metastore in " + - s"Spark SQL specific format, which is NOT compatible with Hive." - (None, message) - } + tableDefinition.copy( + storage = tableDefinition.storage.copy( + locationUri = location, + inputFormat = serde.inputFormat, + outputFormat = serde.outputFormat, + serde = serde.serde, + properties = storagePropsWithLocation + ), + properties = tableDefinition.properties ++ tableProperties) + } - (hiveCompatibleTable, logMessage) match { - case (Some(table), message) => - // We first try to save the metadata of the table in a Hive compatible way. - // If Hive throws an error, we fall back to save its metadata in the Spark SQL - // specific way. - try { - logInfo(message) - saveTableIntoHive(table, ignoreIfExists) - } catch { - case NonFatal(e) => - val warningMessage = - s"Could not persist ${tableDefinition.identifier.quotedString} in a Hive " + - "compatible way. Persisting it into Hive metastore in Spark SQL specific format." - logWarning(warningMessage, e) - saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) - } + val qualifiedTableName = tableDefinition.identifier.quotedString + val maybeSerde = HiveSerDe.sourceToSerDe(tableDefinition.provider.get) + val skipHiveMetadata = tableDefinition.storage.properties + .getOrElse("skipHiveMetadata", "false").toBoolean + + val (hiveCompatibleTable, logMessage) = maybeSerde match { + case _ if skipHiveMetadata => + val message = + s"Persisting data source table $qualifiedTableName into Hive metastore in" + + "Spark SQL specific format, which is NOT compatible with Hive." + (None, message) + + // our bucketing is un-compatible with hive(different hash function) + case _ if tableDefinition.bucketSpec.nonEmpty => + val message = + s"Persisting bucketed data source table $qualifiedTableName into " + + "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " + (None, message) + + case Some(serde) => + val message = + s"Persisting file based data source table $qualifiedTableName into " + + s"Hive metastore in Hive compatible format." + (Some(newHiveCompatibleMetastoreTable(serde)), message) + + case _ => + val provider = tableDefinition.provider.get + val message = + s"Couldn't find corresponding Hive SerDe for data source provider $provider. " + + s"Persisting data source table $qualifiedTableName into Hive metastore in " + + s"Spark SQL specific format, which is NOT compatible with Hive." + (None, message) + } - case (None, message) => - logWarning(message) - saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) - } + (hiveCompatibleTable, logMessage) match { + case (Some(table), message) => + // We first try to save the metadata of the table in a Hive compatible way. + // If Hive throws an error, we fall back to save its metadata in the Spark SQL + // specific way. + try { + logInfo(message) + saveTableIntoHive(table, ignoreIfExists) + } catch { + case NonFatal(e) => + val warningMessage = + s"Could not persist ${tableDefinition.identifier.quotedString} in a Hive " + + "compatible way. Persisting it into Hive metastore in Spark SQL specific format." + logWarning(warningMessage, e) + saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) + } + + case (None, message) => + logWarning(message) + saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) } } @@ -789,7 +804,21 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = withClient { requireTableExists(db, table) - val lowerCasedParts = parts.map(p => p.copy(spec = lowerCasePartitionSpec(p.spec))) + + val tableMeta = getTable(db, table) + val partitionColumnNames = tableMeta.partitionColumnNames + val tablePath = new Path(tableMeta.location) + val partsWithLocation = parts.map { p => + // Ideally we can leave the partition location empty and let Hive metastore to set it. + // However, Hive metastore is not case preserving and will generate wrong partition location + // with lower cased partition column names. Here we set the default partition location + // manually to avoid this problem. + val partitionPath = p.storage.locationUri.map(new Path(_)).getOrElse { + ExternalCatalogUtils.generatePartitionPath(p.spec, partitionColumnNames, tablePath) + } + p.copy(storage = p.storage.copy(locationUri = Some(partitionPath.toString))) + } + val lowerCasedParts = partsWithLocation.map(p => p.copy(spec = lowerCasePartitionSpec(p.spec))) client.createPartitions(db, table, lowerCasedParts, ignoreIfExists) } @@ -810,6 +839,24 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat newSpecs: Seq[TablePartitionSpec]): Unit = withClient { client.renamePartitions( db, table, specs.map(lowerCasePartitionSpec), newSpecs.map(lowerCasePartitionSpec)) + + val tableMeta = getTable(db, table) + val partitionColumnNames = tableMeta.partitionColumnNames + // Hive metastore is not case preserving and keeps partition columns with lower cased names. + // When Hive rename partition for managed tables, it will update the partition location with + // a default value generate by the new spec and lower cased partition column names. This is + // unexpected and we need to call `alterPartitions` to fix the partition location. + val hasUpperCasePartitionColumn = partitionColumnNames.exists(col => col.toLowerCase != col) + if (tableMeta.tableType == MANAGED && hasUpperCasePartitionColumn) { + val tablePath = new Path(tableMeta.location) + val newParts = newSpecs.map { spec => + val partition = client.getPartition(db, table, lowerCasePartitionSpec(spec)) + val partitionPath = ExternalCatalogUtils.generatePartitionPath( + spec, partitionColumnNames, tablePath) + partition.copy(storage = partition.storage.copy(locationUri = Some(partitionPath.toString))) + } + alterPartitions(db, table, newParts) + } } override def alterPartitions( @@ -817,6 +864,11 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat table: String, newParts: Seq[CatalogTablePartition]): Unit = withClient { val lowerCasedParts = newParts.map(p => p.copy(spec = lowerCasePartitionSpec(p.spec))) + // Note: Before altering table partitions in Hive, you *must* set the current database + // to the one that contains the table of interest. Otherwise you will end up with the + // most helpful error message ever: "Unable to alter partition. alter is not possible." + // See HIVE-2742 for more detail. + client.setCurrentDatabase(db) client.alterPartitions(db, table, lowerCasedParts) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index d3873cf6c823..fbd705172cae 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -445,7 +445,7 @@ object SetWarehouseLocationTest extends Logging { catalog.getTableMetadata(TableIdentifier("testLocation", Some("default"))) val expectedLocation = "file:" + expectedWarehouseLocation.toString + "/testlocation" - val actualLocation = tableMetadata.storage.locationUri.get + val actualLocation = tableMetadata.location if (actualLocation != expectedLocation) { throw new Exception( s"Expected table location is $expectedLocation. But, it is actually $actualLocation") @@ -461,7 +461,7 @@ object SetWarehouseLocationTest extends Logging { catalog.getTableMetadata(TableIdentifier("testLocation", Some("testLocationDB"))) val expectedLocation = "file:" + expectedWarehouseLocation.toString + "/testlocationdb.db/testlocation" - val actualLocation = tableMetadata.storage.locationUri.get + val actualLocation = tableMetadata.location if (actualLocation != expectedLocation) { throw new Exception( s"Expected table location is $expectedLocation. But, it is actually $actualLocation") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index cfc1d81d544e..9f4401ae2256 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -29,7 +29,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle val expectedPath = spark.sharedState.externalCatalog.getDatabase(dbName).locationUri + "/" + tableName - assert(metastoreTable.storage.locationUri.get === expectedPath) + assert(metastoreTable.location === expectedPath) } private def getTableNames(dbName: Option[String] = None): Array[String] = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 4150e649bef8..eedcef3995ce 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -426,7 +426,7 @@ class HiveDDLSuite sql("CREATE TABLE tab1 (height INT, length INT) PARTITIONED BY (a INT, b INT)") val part1 = Map("a" -> "1", "b" -> "5") val part2 = Map("a" -> "2", "b" -> "6") - val root = new Path(catalog.getTableMetadata(tableIdent).storage.locationUri.get) + val root = new Path(catalog.getTableMetadata(tableIdent).location) val fs = root.getFileSystem(spark.sparkContext.hadoopConfiguration) // valid fs.mkdirs(new Path(new Path(root, "a=1"), "b=5")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 28e5dffb1152..1cce23f918d0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -542,7 +542,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } userSpecifiedLocation match { case Some(location) => - assert(r.catalogTable.storage.locationUri.get === location) + assert(r.catalogTable.location === location) case None => // OK. } // Also make sure that the format and serde are as desired. From bbe1e1224907f44ff384e42aa7f65b5472d9194a Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 7 Nov 2016 18:32:23 -0800 Subject: [PATCH 07/27] more test --- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 6 ++-- .../datasources/DataSourceStrategy.scala | 1 - .../PartitionProviderCompatibilitySuite.scala | 28 +++++++++++++++++++ 4 files changed, 32 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 072522ce3f23..c912d79c7f55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -184,7 +184,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { UnresolvedRelation(tableIdent, None), partitionKeys, query, - OverwriteOptions(overwrite, staticPartitionKeys), + OverwriteOptions(overwrite, if (overwrite) staticPartitionKeys else Map.empty), ctx.EXISTS != null) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 75baba887924..574caf039d3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -357,9 +357,9 @@ case class BroadcastHint(child: LogicalPlan) extends UnaryNode { case class OverwriteOptions( enabled: Boolean, staticPartitionKeys: CatalogTypes.TablePartitionSpec = Map.empty) { -// if (staticPartitionKeys.nonEmpty) { -// assert(enabled, "Overwrite must be enabled when specifying specific partitions.") -// } + if (staticPartitionKeys.nonEmpty) { + assert(enabled, "Overwrite must be enabled when specifying specific partitions.") + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 37eb3aaa060e..05e9c1397d67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -146,7 +146,6 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { // values in the PARTITION clause (e.g. b in the above example). // dynamic_partitioning_columns are partitioning columns that do not assigned // values in the PARTITION clause (e.g. c in the above example). - // TODO(ekl) get rid of this and combine with static partition key arg for overwrite case insert @ logical.InsertIntoTable( relation @ LogicalRelation(t: HadoopFsRelation, _, _), parts, query, overwrite, false) if query.resolved && parts.exists(_._2.isDefined) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index 0f145f74de87..66ba64d0255e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -246,6 +246,34 @@ class PartitionProviderCompatibilitySuite assert(spark.sql("select * from test").count() == 243) spark.sql("insert overwrite table test select id, 1, 1 from range(10)") assert(spark.sql("select * from test").count() == 10) + + // dynamic append to custom location + withTempDir { a => + spark.sql( + s"alter table test partition (p1=1, p2=2) set location '${a.getAbsolutePath}'" + ).count() + spark.sql("insert into test partition (p1=1, p2) select id, id from range(100)") + spark.sql("insert into test partition (p1=1, p2) select id, id from range(100)") + assert(spark.sql("select * from test where p1=1").count() == 210) + assert(spark.sql("select * from test where p1=1 and p2=2").count() == 2) + } + sql("refresh table test") + assert(spark.sql("select * from test where p1=1 and p2=2").count() == 0) + + // dynamic overwrite of custom locations + withTempDir { a => + spark.sql( + s"alter table test partition (p1=1, p2=2) set location '${a.getAbsolutePath}'" + ).count() + spark.sql( + "insert overwrite table test partition (p1=1, p2) select id, id from range(100)") + spark.sql( + "insert overwrite table test partition (p1=1, p2) select id, id from range(100)") + assert(spark.sql("select * from test where p1=1").count() == 100) + assert(spark.sql("select * from test where p1=1 and p2=2").count() == 1) + } + sql("refresh table test") + assert(spark.sql("select * from test where p1=1 and p2=2").count() == 0) } } } From a905b093c1e2cee0b81c2310a41558e5d87e4a87 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 7 Nov 2016 19:25:37 -0800 Subject: [PATCH 08/27] Mon Nov 7 19:25:37 PST 2016 --- .../datasources/DataSourceStrategy.scala | 57 ++++++++++++------- .../InsertIntoHadoopFsRelationCommand.scala | 4 +- .../PartitionProviderCompatibilitySuite.scala | 14 ++++- 3 files changed, 50 insertions(+), 25 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 05e9c1397d67..c5af995eea3a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SimpleCatalogRelation} +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTablePartition, SimpleCatalogRelation} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} -import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, DDLUtils, ExecutedCommandExec} +import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -182,28 +182,43 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { "Cannot overwrite a path that is also being read from.") } - val customPartitionLocations: Map[TablePartitionSpec, String] = - if (t.sparkSession.sessionState.conf.manageFilesourcePartitions && - l.catalogTable.get.tracksPartitionsInCatalog) { - getCustomPartitionLocations( - t.sparkSession, l.catalogTable.get, outputPath, overwrite.staticPartitionKeys) - } else { - Map.empty - } - val partitionSchema = query.resolve( t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver) - // TODO(ekl) move this into InsertIntoHadoopFsRelationCommand + var allStaticPartitions: Seq[TablePartitionSpec] = Nil + var customPartitionLocations: Map[TablePartitionSpec, String] = Map.empty + + // When partitions are tracked by the catalog, compute all custom partition locations that + // may be relevant to the insertion job. + if (t.sparkSession.sessionState.conf.manageFilesourcePartitions && + l.catalogTable.get.tracksPartitionsInCatalog) { + val matchingPartitions = t.sparkSession.sessionState.catalog.listPartitions( + l.catalogTable.get.identifier, Some(overwrite.staticPartitionKeys)) + allStaticPartitions = matchingPartitions.map(_.spec) + customPartitionLocations = getCustomPartitionLocations( + t.sparkSession, l.catalogTable.get, outputPath, matchingPartitions) + } + + // Callback for updating metastore partition metadata after the insertion job completes. + // TODO(ekl) consider moving this into InsertIntoHadoopFsRelationCommand def refreshPartitionsCallback(updatedPartitions: Seq[TablePartitionSpec]): Unit = { - if (l.catalogTable.isDefined && updatedPartitions.nonEmpty && - l.catalogTable.get.partitionColumnNames.nonEmpty && + if (l.catalogTable.isDefined && l.catalogTable.get.partitionColumnNames.nonEmpty && + t.sparkSession.sessionState.conf.manageFilesourcePartitions && l.catalogTable.get.tracksPartitionsInCatalog) { - val metastoreUpdater = AlterTableAddPartitionCommand( - l.catalogTable.get.identifier, - updatedPartitions.map(p => (p, None)), - ifNotExists = true) - metastoreUpdater.run(t.sparkSession) + if (updatedPartitions.nonEmpty) { + AlterTableAddPartitionCommand( + l.catalogTable.get.identifier, + updatedPartitions.map(p => (p, None)), + ifNotExists = true).run(t.sparkSession) + } + if (overwrite.enabled) { + val deletedPartitions = allStaticPartitions.toSet -- updatedPartitions + if (deletedPartitions.nonEmpty) { + AlterTableDropPartitionCommand( + l.catalogTable.get.identifier, deletedPartitions.toSeq, + ifExists = true, purge = true).run(t.sparkSession) + } + } } t.location.refresh() } @@ -227,12 +242,10 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { spark: SparkSession, table: CatalogTable, basePath: Path, - staticPartitionKeys: TablePartitionSpec): Map[TablePartitionSpec, String] = { + partitions: Seq[CatalogTablePartition]): Map[TablePartitionSpec, String] = { val hadoopConf = spark.sessionState.newHadoopConf val fs = basePath.getFileSystem(hadoopConf) val qualifiedBasePath = basePath.makeQualified(fs.getUri, fs.getWorkingDirectory) - val partitions = spark.sessionState.catalog.listPartitions( - table.identifier, Some(staticPartitionKeys)) partitions.flatMap { p => val defaultLocation = qualifiedBasePath.suffix( "/" + PartitioningUtils.getPathFragment(p.spec, table.partitionSchema)).toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 726ab2966715..b7faca9db71b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -32,6 +32,9 @@ import org.apache.spark.sql.execution.command.RunnableCommand /** * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. * Writing to dynamic partitions is also supported. + * + * @param staticPartitionKeys + * @param customPartitionLocations */ case class InsertIntoHadoopFsRelationCommand( outputPath: Path, @@ -109,7 +112,6 @@ case class InsertIntoHadoopFsRelationCommand( * Deletes all partition files that match the specified static prefix. Partitions with custom * locations are also cleared based on the custom locations map given to this class. */ - // TODO(ekl) also delete partition metadata from the catalog private def deleteMatchingPartitions(fs: FileSystem, qualifiedOutputPath: Path): Unit = { val staticPartitionPrefix = if (staticPartitionKeys.nonEmpty) { "/" + partitionColumns.flatMap { p => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index 66ba64d0255e..3c83736ddd3d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -220,22 +220,32 @@ class PartitionProviderCompatibilitySuite // dynamic append with full spec, new dir spark.sql("insert into test partition (p1=100, p2=100) select id from range(10)") assert(spark.sql("select * from test where p1=100").count() == 20) + assert(spark.sql("show partitions test").count() == 25) // dynamic overwrite with partial spec, existing dir spark.sql( "insert overwrite table test partition (p1=1, p2) select id, id from range(100)") assert(spark.sql("select * from test where p1=1").count() == 100) + assert(spark.sql("show partitions test").count() == 115) // dynamic overwrite with full spec, existing dir spark.sql( "insert overwrite table test partition (p1=1, p2=1) select id from range(100)") assert(spark.sql("select * from test where p1=1").count() == 199) assert(spark.sql("select * from test where p1=1 and p2=1").count() == 100) + assert(spark.sql("show partitions test").count() == 115) // dynamic overwrite with partial spec, new dir spark.sql( "insert overwrite table test partition (p1=500, p2) select id, id from range(10)") assert(spark.sql("select * from test where p1=500").count() == 10) + assert(spark.sql("show partitions test").count() == 125) + + // dynamic overwrite with partial spec again (test partition cleanup) + spark.sql( + "insert overwrite table test partition (p1=1, p2) select id, id from range(10)") + assert(spark.sql("select * from test where p1=1").count() == 10) + assert(spark.sql("show partitions test").count() == 35) // dynamic overwrite with full spec, new dir spark.sql( @@ -243,14 +253,14 @@ class PartitionProviderCompatibilitySuite assert(spark.sql("select * from test where p1=500 and p2=500").count() == 10) // overwrite entire table - assert(spark.sql("select * from test").count() == 243) spark.sql("insert overwrite table test select id, 1, 1 from range(10)") assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 1) // dynamic append to custom location withTempDir { a => spark.sql( - s"alter table test partition (p1=1, p2=2) set location '${a.getAbsolutePath}'" + s"alter table test add partition (p1=1, p2=2) location '${a.getAbsolutePath}'" ).count() spark.sql("insert into test partition (p1=1, p2) select id, id from range(100)") spark.sql("insert into test partition (p1=1, p2) select id, id from range(100)") From 3c43fa58763e39cd9acddcdcb165f7823bd52b2f Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 7 Nov 2016 19:28:49 -0800 Subject: [PATCH 09/27] Mon Nov 7 19:28:49 PST 2016 --- .../spark/internal/io/HadoopMapReduceCommitProtocol.scala | 6 +++--- .../sql/execution/datasources/DataSourceStrategy.scala | 7 +++++++ 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 154a0ee4c329..9deae14e5fe8 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -49,7 +49,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) * OutputCommitter, we must manually move these to their final locations on task commit. * TODO(ekl) it would be nice to provide better atomicity for this type of output. */ - @transient private var taskAbsPathOutputs: ArrayBuffer[String] = _ + @transient private var addedExternalFiles: ArrayBuffer[String] = _ protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { context.getOutputFormatClass.newInstance().getOutputCommitter(context) @@ -76,7 +76,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) absoluteDir match { case Some(d) => val absOutputPath = new Path(d, filename).toString - taskAbsPathOutputs += absOutputPath + addedExternalFiles += absOutputPath absOutputPath case _ => relativeDir match { @@ -117,7 +117,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) override def setupTask(taskContext: TaskAttemptContext): Unit = { committer = setupCommitter(taskContext) committer.setupTask(taskContext) - taskAbsPathOutputs = new ArrayBuffer[String] + addedExternalFiles = new ArrayBuffer[String] } override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index c5af995eea3a..a680fb521379 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -238,6 +238,13 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { insertCmd } + /** + * Given a set of input partitions, returns those that have locations that differ from the + * Hive default (e.g. /k1=v1/k2=v2). These partitions were manually assigned locations by + * the user. + * + * @return a mapping from partition specs to their custom locations + */ private def getCustomPartitionLocations( spark: SparkSession, table: CatalogTable, From 13aa4814a34669fd99b6a6290855abd064449b6f Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 7 Nov 2016 19:34:38 -0800 Subject: [PATCH 10/27] Mon Nov 7 19:34:38 PST 2016 --- .../InsertIntoHadoopFsRelationCommand.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index b7faca9db71b..0d951148948b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -33,8 +33,14 @@ import org.apache.spark.sql.execution.command.RunnableCommand * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. * Writing to dynamic partitions is also supported. * - * @param staticPartitionKeys - * @param customPartitionLocations + * @param staticPartitionKeys partial partitioning spec for write. This defines the scope of + * partition overwrites: when the spec is empty, all partitions are + * overwritten. When it covers a prefix of the partition keys, only + * partitions matching the prefix are overwritten. + * @param customPartitionLocations mapping of partition specs to their custom locations. The + * caller should guarantee that exactly those table partitions + * falling under the specified static partition keys are contained + * in this map, and that no other partitions are. */ case class InsertIntoHadoopFsRelationCommand( outputPath: Path, @@ -43,7 +49,7 @@ case class InsertIntoHadoopFsRelationCommand( partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], fileFormat: FileFormat, - refreshFunction: (Seq[TablePartitionSpec]) => Unit, + refreshFunction: Seq[TablePartitionSpec] => Unit, options: Map[String, String], @transient query: LogicalPlan, mode: SaveMode) From 37afbb103b24fced17efa52c6ed4aca21b746df2 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 7 Nov 2016 20:02:13 -0800 Subject: [PATCH 11/27] Mon Nov 7 20:02:13 PST 2016 --- .../internal/io/FileCommitProtocol.scala | 23 +++++---- .../io/HadoopMapReduceCommitProtocol.scala | 48 +++++++++---------- .../datasources/FileFormatWriter.scala | 5 +- .../ManifestFileCommitProtocol.scala | 9 +--- 4 files changed, 42 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala index 1c4382fa8308..ef0fa2aa623f 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -80,18 +80,23 @@ abstract class FileCommitProtocol { * 4. bucket id * 5. source specific file extension, e.g. ".snappy.parquet" * - * The directory parameters specify 2, and "ext" parameter specifies both 4 and 5, and the rest + * The "dir" parameter specifies 2, and "ext" parameter specifies both 4 and 5, and the rest * are left to the commit protocol implementation to decide. + */ + def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String + + /** + * Similar to newTaskTempFile(), but allows files to committed to an absolute output location + * instead of a relative directory under some base path controlled by the committer. Depending + * on the protocol implementation, files added this way may have weaker atomicity guarantees. * - * @param relativeDir relative path of the final output directory from the base path, or None - * if the file should be placed in the base path or an absolute path. - * @param absoluteDir absolute path of the final output directory. If specified, the base path - * will be ignored. This may not be specified if relativeDir is. - * @param ext extension of the final output file. + * Implementing this is optional. */ - def newTaskTempFile( - taskContext: TaskAttemptContext, - relativeDir: Option[String], absoluteDir: Option[String], ext: String): String + def newTaskTempFileAbsPath( + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { + throw new UnsupportedOperationException( + s"$this does not support adding files with an absolute path") + } /** * Commits a task after the writes succeed. Must be called on the executors when running tasks. diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 9deae14e5fe8..983bd33cbf50 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -49,23 +49,15 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) * OutputCommitter, we must manually move these to their final locations on task commit. * TODO(ekl) it would be nice to provide better atomicity for this type of output. */ - @transient private var addedExternalFiles: ArrayBuffer[String] = _ + @transient private var addedAbsPathFiles: ArrayBuffer[String] = _ protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { context.getOutputFormatClass.newInstance().getOutputCommitter(context) } override def newTaskTempFile( - taskContext: TaskAttemptContext, - relativeDir: Option[String], absoluteDir: Option[String], ext: String): String = { - if (absoluteDir.isDefined) { - require(!relativeDir.isDefined, "Cannot specify both abs and relative output dirs.") - } - // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet - // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, - // the file name is fine and won't overflow. - val split = taskContext.getTaskAttemptID.getTaskID.getId - val filename = f"part-$split%05d-$jobId$ext" + taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { + val filename = getFilename(taskContext, ext) val stagingDir: String = committer match { // For FileOutputCommitter it has its own staging path called "work path". @@ -73,21 +65,29 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) case _ => path } - absoluteDir match { - case Some(d) => - val absOutputPath = new Path(d, filename).toString - addedExternalFiles += absOutputPath - absOutputPath - case _ => - relativeDir match { - case Some(d) => - new Path(new Path(stagingDir, d), filename).toString - case _ => - new Path(stagingDir, filename).toString - } + dir.map { d => + new Path(new Path(stagingDir, d), filename).toString + }.getOrElse { + new Path(stagingDir, filename).toString } } + override def newTaskTempFileAbsPath( + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { + val filename = getFilename(taskContext, ext) + val absOutputPath = new Path(absoluteDir, filename).toString + addedAbsPathFiles += absOutputPath + absOutputPath + } + + private def getFilename(taskContext: TaskAttemptContext, ext: String): String = { + // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet + // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, + // the file name is fine and won't overflow. + val split = taskContext.getTaskAttemptID.getTaskID.getId + f"part-$split%05d-$jobId$ext" + } + override def setupJob(jobContext: JobContext): Unit = { // Setup IDs val jobId = SparkHadoopWriter.createJobID(new Date, 0) @@ -117,7 +117,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) override def setupTask(taskContext: TaskAttemptContext): Unit = { committer = setupCommitter(taskContext) committer.setupTask(taskContext) - addedExternalFiles = new ArrayBuffer[String] + addedAbsPathFiles = new ArrayBuffer[String] } override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index e5a9ea8d9fb3..2091465303e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -230,7 +230,6 @@ object FileFormatWriter extends Logging { val tmpFilePath = committer.newTaskTempFile( taskAttemptContext, None, - None, description.outputWriterFactory.getFileExtension(taskAttemptContext)) val outputWriter = description.outputWriterFactory.newInstance( @@ -322,9 +321,9 @@ object FileFormatWriter extends Logging { None } val path = if (customPath.isDefined) { - committer.newTaskTempFile(taskAttemptContext, None, customPath, ext) + committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) } else { - committer.newTaskTempFile(taskAttemptContext, partDir, None, ext) + committer.newTaskTempFile(taskAttemptContext, partDir, ext) } val newWriter = description.outputWriterFactory.newInstance( path = path, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index 03c762813b90..1fe13fa1623f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -78,12 +78,7 @@ class ManifestFileCommitProtocol(jobId: String, path: String) } override def newTaskTempFile( - taskContext: TaskAttemptContext, - relativeDir: Option[String], absoluteDir: Option[String], ext: String): String = { - if (absoluteDir.isDefined) { - throw new UnsupportedOperationException( - "absolute output path not supported by manifest commit protocol") - } + taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, // the file name is fine and won't overflow. @@ -91,7 +86,7 @@ class ManifestFileCommitProtocol(jobId: String, path: String) val uuid = UUID.randomUUID.toString val filename = f"part-$split%05d-$uuid$ext" - val file = relativeDir.map { d => + val file = dir.map { d => new Path(new Path(path, d), filename).toString }.getOrElse { new Path(path, filename).toString From dddee47b6f75da53beb1f99f1d2c6839444b0dd7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 8 Nov 2016 13:47:59 +0800 Subject: [PATCH 12/27] use Path --- .../sql/catalyst/catalog/ExternalCatalogSuite.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index bc9bcccdb961..44684c13fea5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -652,10 +652,11 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac // -------------------------------------------------------------------------- private def exists(uri: String, children: String*): Boolean = { - val base = new File(new URI(uri)) - children.foldLeft(base) { - case (parent, child) => new File(parent, child) - }.exists() + val base = new Path(uri) + val finalPath = children.foldLeft(base) { + case (parent, child) => new Path(parent, child) + } + base.getFileSystem(new Configuration()).exists(finalPath) } test("create/drop database should create/delete the directory") { From e97c17f9341b81456f51612b61049b47b866a14c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 8 Nov 2016 17:19:57 +0800 Subject: [PATCH 13/27] address comments --- .../catalog/ExternalCatalogUtils.scala | 29 +++++++++++++++ .../sql/catalyst/catalog/interface.scala | 1 + .../spark/sql/execution/command/ddl.scala | 6 ++-- .../datasources/PartitioningUtils.scala | 36 +++---------------- 4 files changed, 38 insertions(+), 34 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala index 5d17189fc852..b1442eec164d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -72,6 +72,35 @@ object ExternalCatalogUtils { builder.toString() } + + def unescapePathName(path: String): String = { + val sb = new StringBuilder + var i = 0 + + while (i < path.length) { + val c = path.charAt(i) + if (c == '%' && i + 2 < path.length) { + val code: Int = try { + Integer.parseInt(path.substring(i + 1, i + 3), 16) + } catch { + case _: Exception => -1 + } + if (code >= 0) { + sb.append(code.asInstanceOf[Char]) + i += 3 + } else { + sb.append(c) + i += 1 + } + } else { + sb.append(c) + i += 1 + } + } + + sb.toString() + } + def generatePartitionPath( spec: TablePartitionSpec, partitionColumnNames: Seq[String], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index b3b84eaadbb7..0944db51243d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -171,6 +171,7 @@ case class CatalogTable( throw new AnalysisException(s"table $identifier did not specify database") } + /** Return the table location, assuming it is specified. */ def location: String = storage.locationUri.getOrElse { throw new AnalysisException(s"table $identifier did not specify locationUri") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index c2862b4d73e2..84a63fdb9f36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.Resolver -import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTablePartition, CatalogTableType, SessionCatalog} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.execution.datasources.{CaseInsensitiveMap, PartitioningUtils} @@ -558,9 +558,9 @@ case class AlterTableRecoverPartitionsCommand( val name = st.getPath.getName if (st.isDirectory && name.contains("=")) { val ps = name.split("=", 2) - val columnName = PartitioningUtils.unescapePathName(ps(0)) + val columnName = ExternalCatalogUtils.unescapePathName(ps(0)) // TODO: Validate the value - val value = PartitioningUtils.unescapePathName(ps(1)) + val value = ExternalCatalogUtils.unescapePathName(ps(1)) if (resolver(columnName, partitionNames.head)) { scanPartitions(spark, fs, filter, st.getPath, spec ++ Map(partitionNames.head -> value), partitionNames.drop(1), threshold, resolver) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index abb5ec558474..a28b04ca3fb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.types._ @@ -62,6 +61,9 @@ object PartitioningUtils { require(columnNames.size == literals.size) } + import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.DEFAULT_PARTITION_NAME + import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName + /** * Given a group of qualified paths, tries to parse them and returns a partition specification. * For example, given: @@ -373,14 +375,14 @@ object PartitioningUtils { .orElse(Try(Literal(JTimestamp.valueOf(unescapePathName(raw))))) // Then falls back to string .getOrElse { - if (raw == ExternalCatalogUtils.DEFAULT_PARTITION_NAME) { + if (raw == DEFAULT_PARTITION_NAME) { Literal.create(null, NullType) } else { Literal.create(unescapePathName(raw), StringType) } } } else { - if (raw == ExternalCatalogUtils.DEFAULT_PARTITION_NAME) { + if (raw == DEFAULT_PARTITION_NAME) { Literal.create(null, NullType) } else { Literal.create(unescapePathName(raw), StringType) @@ -443,32 +445,4 @@ object PartitioningUtils { Literal.create(Cast(l, desiredType).eval(), desiredType) } } - - def unescapePathName(path: String): String = { - val sb = new StringBuilder - var i = 0 - - while (i < path.length) { - val c = path.charAt(i) - if (c == '%' && i + 2 < path.length) { - val code: Int = try { - Integer.parseInt(path.substring(i + 1, i + 3), 16) - } catch { - case _: Exception => -1 - } - if (code >= 0) { - sb.append(code.asInstanceOf[Char]) - i += 3 - } else { - sb.append(c) - i += 1 - } - } else { - sb.append(c) - i += 1 - } - } - - sb.toString() - } } From f85bb27d9a68df3dfa8dd7f241446f327b1d919d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 8 Nov 2016 17:47:27 +0800 Subject: [PATCH 14/27] fix bug --- .../spark/sql/hive/HiveExternalCatalog.scala | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 9d18c104c651..897dee8696d6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive +import java.io.IOException import java.util import scala.util.control.NonFatal @@ -26,7 +27,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.thrift.TException -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier @@ -843,17 +844,25 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat val tableMeta = getTable(db, table) val partitionColumnNames = tableMeta.partitionColumnNames // Hive metastore is not case preserving and keeps partition columns with lower cased names. - // When Hive rename partition for managed tables, it will update the partition location with - // a default value generate by the new spec and lower cased partition column names. This is - // unexpected and we need to call `alterPartitions` to fix the partition location. + // When Hive rename partition for managed tables, it will create the partition location with + // a default path generate by the new spec with lower cased partition column names. This is + // unexpected and we need to rename them manually and alter the partition location. val hasUpperCasePartitionColumn = partitionColumnNames.exists(col => col.toLowerCase != col) if (tableMeta.tableType == MANAGED && hasUpperCasePartitionColumn) { val tablePath = new Path(tableMeta.location) + val fs = tablePath.getFileSystem(hadoopConf) val newParts = newSpecs.map { spec => val partition = client.getPartition(db, table, lowerCasePartitionSpec(spec)) - val partitionPath = ExternalCatalogUtils.generatePartitionPath( + val wrongPath = new Path(partition.storage.locationUri.get) + val rightPath = ExternalCatalogUtils.generatePartitionPath( spec, partitionColumnNames, tablePath) - partition.copy(storage = partition.storage.copy(locationUri = Some(partitionPath.toString))) + try { + fs.rename(wrongPath, rightPath) + } catch { + case e: IOException => + throw new SparkException(s"Unable to rename partition path $wrongPath", e) + } + partition.copy(storage = partition.storage.copy(locationUri = Some(rightPath.toString))) } alterPartitions(db, table, newParts) } From 6dc4cb5f13a0d75e89282e690591bcc5e0cd6cb9 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 8 Nov 2016 11:29:45 -0800 Subject: [PATCH 15/27] Tue Nov 8 11:29:45 PST 2016 --- .../internal/io/FileCommitProtocol.scala | 7 ++--- .../io/HadoopMapReduceCommitProtocol.scala | 31 +++++++++++++------ 2 files changed, 24 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala index ef0fa2aa623f..19746143b67e 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -86,11 +86,8 @@ abstract class FileCommitProtocol { def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String /** - * Similar to newTaskTempFile(), but allows files to committed to an absolute output location - * instead of a relative directory under some base path controlled by the committer. Depending - * on the protocol implementation, files added this way may have weaker atomicity guarantees. - * - * Implementing this is optional. + * Similar to newTaskTempFile(), but allows files to committed to an absolute output location. + * Depending on the implementation, there may be weaker guarantees around adding files this way. */ def newTaskTempFileAbsPath( taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 983bd33cbf50..565753a5b98a 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -19,7 +19,7 @@ package org.apache.spark.internal.io import java.util.Date -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ @@ -45,11 +45,10 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) @transient private var committer: OutputCommitter = _ /** - * Tracks files staged by this task for absolute output paths. Unlike files staged by the - * OutputCommitter, we must manually move these to their final locations on task commit. - * TODO(ekl) it would be nice to provide better atomicity for this type of output. + * Tracks files staged by this task for absolute output paths. These outputs are not managed by + * the Hadoop OutputCommitter, so we must move these to their final locations on job commit. */ - @transient private var addedAbsPathFiles: ArrayBuffer[String] = _ + @transient private var addedAbsPathFiles: mutable.Map[String, String] = null protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { context.getOutputFormatClass.newInstance().getOutputCommitter(context) @@ -76,8 +75,9 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { val filename = getFilename(taskContext, ext) val absOutputPath = new Path(absoluteDir, filename).toString - addedAbsPathFiles += absOutputPath - absOutputPath + val tmpOutputPath = new Path(path, "_temporary-" + jobId + "/" + filename).toString + addedAbsPathFiles(tmpOutputPath) = absOutputPath + tmpOutputPath } private def getFilename(taskContext: TaskAttemptContext, ext: String): String = { @@ -107,7 +107,15 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) } override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { + logDebug(s"Committing staged files") committer.commitJob(jobContext) + val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]]).reduce(_ ++ _) + logDebug(s"Committing staged files with absolute locations $filesToMove") + for ((src, dst) <- filesToMove) { + val tmp = new Path(src) + val fs = tmp.getFileSystem(jobContext.getConfiguration) + fs.rename(tmp, new Path(dst)) + } } override def abortJob(jobContext: JobContext): Unit = { @@ -117,17 +125,22 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) override def setupTask(taskContext: TaskAttemptContext): Unit = { committer = setupCommitter(taskContext) committer.setupTask(taskContext) - addedAbsPathFiles = new ArrayBuffer[String] + addedAbsPathFiles = mutable.Map[String, String]() } override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { val attemptId = taskContext.getTaskAttemptID SparkHadoopMapRedUtil.commitTask( committer, taskContext, attemptId.getJobID.getId, attemptId.getTaskID.getId) - EmptyTaskCommitMessage + new TaskCommitMessage(addedAbsPathFiles.toMap) } override def abortTask(taskContext: TaskAttemptContext): Unit = { committer.abortTask(taskContext) + // best effort cleanup of other staged files + for ((src, _) <- addedAbsPathFiles) { + val tmp = new Path(src) + tmp.getFileSystem(taskContext.getConfiguration).delete(tmp, false) + } } } From d68d1db65ec242fa08d2124255e74efaa90c3f67 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 8 Nov 2016 11:45:44 -0800 Subject: [PATCH 16/27] Tue Nov 8 11:45:44 PST 2016 --- .../io/HadoopMapReduceCommitProtocol.scala | 12 ++++++---- .../datasources/DataSourceStrategy.scala | 23 ++++++++++--------- 2 files changed, 20 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 565753a5b98a..995e8332e730 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -50,6 +50,8 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) */ @transient private var addedAbsPathFiles: mutable.Map[String, String] = null + private def absPathStagingDir: Path = new Path(path, "_temporary-" + jobId) + protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { context.getOutputFormatClass.newInstance().getOutputCommitter(context) } @@ -75,7 +77,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { val filename = getFilename(taskContext, ext) val absOutputPath = new Path(absoluteDir, filename).toString - val tmpOutputPath = new Path(path, "_temporary-" + jobId + "/" + filename).toString + val tmpOutputPath = new Path(absPathStagingDir, filename).toString addedAbsPathFiles(tmpOutputPath) = absOutputPath tmpOutputPath } @@ -111,15 +113,17 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) committer.commitJob(jobContext) val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]]).reduce(_ ++ _) logDebug(s"Committing staged files with absolute locations $filesToMove") + val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration) for ((src, dst) <- filesToMove) { - val tmp = new Path(src) - val fs = tmp.getFileSystem(jobContext.getConfiguration) - fs.rename(tmp, new Path(dst)) + fs.rename(new Path(src), new Path(dst)) } + fs.delete(absPathStagingDir, true) } override def abortJob(jobContext: JobContext): Unit = { committer.abortJob(jobContext, JobStatus.State.FAILED) + val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration) + fs.delete(absPathStagingDir, true) } override def setupTask(taskContext: TaskAttemptContext): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index a680fb521379..568ba91d0e5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -184,17 +184,20 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { val partitionSchema = query.resolve( t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver) + val partitionsTrackedByCatalog = + t.sparkSession.sessionState.conf.manageFilesourcePartitions && + l.catalogTable.isDefined && l.catalogTable.get.partitionColumnNames.nonEmpty && + l.catalogTable.get.tracksPartitionsInCatalog - var allStaticPartitions: Seq[TablePartitionSpec] = Nil + var initialMatchingPartitions: Seq[TablePartitionSpec] = Nil var customPartitionLocations: Map[TablePartitionSpec, String] = Map.empty // When partitions are tracked by the catalog, compute all custom partition locations that // may be relevant to the insertion job. - if (t.sparkSession.sessionState.conf.manageFilesourcePartitions && - l.catalogTable.get.tracksPartitionsInCatalog) { + if (partitionsTrackedByCatalog) { val matchingPartitions = t.sparkSession.sessionState.catalog.listPartitions( l.catalogTable.get.identifier, Some(overwrite.staticPartitionKeys)) - allStaticPartitions = matchingPartitions.map(_.spec) + initialMatchingPartitions = matchingPartitions.map(_.spec) customPartitionLocations = getCustomPartitionLocations( t.sparkSession, l.catalogTable.get, outputPath, matchingPartitions) } @@ -202,17 +205,15 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { // Callback for updating metastore partition metadata after the insertion job completes. // TODO(ekl) consider moving this into InsertIntoHadoopFsRelationCommand def refreshPartitionsCallback(updatedPartitions: Seq[TablePartitionSpec]): Unit = { - if (l.catalogTable.isDefined && l.catalogTable.get.partitionColumnNames.nonEmpty && - t.sparkSession.sessionState.conf.manageFilesourcePartitions && - l.catalogTable.get.tracksPartitionsInCatalog) { - if (updatedPartitions.nonEmpty) { + if (partitionsTrackedByCatalog) { + val newPartitions = updatedPartitions.toSet -- initialMatchingPartitions + if (newPartitions.nonEmpty) { AlterTableAddPartitionCommand( - l.catalogTable.get.identifier, - updatedPartitions.map(p => (p, None)), + l.catalogTable.get.identifier, newPartitions.toSeq.map(p => (p, None)), ifNotExists = true).run(t.sparkSession) } if (overwrite.enabled) { - val deletedPartitions = allStaticPartitions.toSet -- updatedPartitions + val deletedPartitions = initialMatchingPartitions.toSet -- updatedPartitions if (deletedPartitions.nonEmpty) { AlterTableDropPartitionCommand( l.catalogTable.get.identifier, deletedPartitions.toSeq, From b5bbb1d664dd23131635c156b6b8628184e0cc4f Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 8 Nov 2016 11:50:51 -0800 Subject: [PATCH 17/27] Tue Nov 8 11:50:51 PST 2016 --- .../spark/internal/io/HadoopMapReduceCommitProtocol.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 995e8332e730..6f8741d67662 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -109,10 +109,9 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) } override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { - logDebug(s"Committing staged files") committer.commitJob(jobContext) val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]]).reduce(_ ++ _) - logDebug(s"Committing staged files with absolute locations $filesToMove") + logDebug(s"Committing files staged for absolute locations $filesToMove") val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration) for ((src, dst) <- filesToMove) { fs.rename(new Path(src), new Path(dst)) From d939b37e6d032ef61f6d0d383f18049ffed7c78a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 8 Nov 2016 23:39:58 +0800 Subject: [PATCH 18/27] work around --- .../catalog/ExternalCatalogSuite.scala | 38 +-- .../spark/sql/hive/HiveExternalCatalog.scala | 237 ++++++++---------- 2 files changed, 120 insertions(+), 155 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 44684c13fea5..c7c653875f4c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.catalyst.catalog -import java.io.File -import java.net.URI - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach @@ -175,23 +172,6 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac } } - test("create managed table without table location") { - val catalog = newBasicCatalog() - val table = CatalogTable( - identifier = TableIdentifier("myTable", Some("db1")), - tableType = CatalogTableType.MANAGED, - storage = storageFormat, - schema = new StructType() - .add("col1", "int") - .add("col2", "string"), - provider = Some("hive")) - catalog.createTable(table, ignoreIfExists = false) - - val tableLocation = new Path(catalog.getTable("db1", "myTable").location) - val defaultTableLocation = new Path(catalog.getDatabase("db1").locationUri, "myTable") - assert(tableLocation == defaultTableLocation) - } - test("drop table") { val catalog = newBasicCatalog() assert(catalog.listTables("db2").toSet == Set("tbl1", "tbl2")) @@ -673,7 +653,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac val catalog = newBasicCatalog() val db = catalog.getDatabase("db1") val table = CatalogTable( - identifier = TableIdentifier("myTable", Some("db1")), + identifier = TableIdentifier("my_table", Some("db1")), tableType = CatalogTableType.MANAGED, storage = CatalogStorageFormat(None, None, None, None, false, Map.empty), schema = new StructType().add("a", "int").add("b", "string"), @@ -681,17 +661,17 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac ) catalog.createTable(table, ignoreIfExists = false) - assert(exists(db.locationUri, "myTable")) + assert(exists(db.locationUri, "my_table")) - catalog.renameTable("db1", "myTable", "yourTable") - assert(!exists(db.locationUri, "myTable")) - assert(exists(db.locationUri, "yourTable")) + catalog.renameTable("db1", "my_table", "your_table") + assert(!exists(db.locationUri, "my_table")) + assert(exists(db.locationUri, "your_table")) - catalog.dropTable("db1", "yourTable", ignoreIfNotExists = false, purge = false) - assert(!exists(db.locationUri, "yourTable")) + catalog.dropTable("db1", "your_table", ignoreIfNotExists = false, purge = false) + assert(!exists(db.locationUri, "your_table")) val externalTable = CatalogTable( - identifier = TableIdentifier("externalTable", Some("db1")), + identifier = TableIdentifier("external_table", Some("db1")), tableType = CatalogTableType.EXTERNAL, storage = CatalogStorageFormat( Some(Utils.createTempDir().getAbsolutePath), @@ -700,7 +680,7 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac provider = Some("hive") ) catalog.createTable(externalTable, ignoreIfExists = false) - assert(!exists(db.locationUri, "externalTable")) + assert(!exists(db.locationUri, "external_table")) } test("create/drop/rename partitions should create/delete/rename the directory") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 897dee8696d6..d7b65eda681b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -197,7 +197,25 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat if (tableDefinition.tableType == VIEW) { client.createTable(tableDefinition, ignoreIfExists) + } else if (tableDefinition.provider.get == DDLUtils.HIVE_PROVIDER) { + // Here we follow data source tables and put table metadata like provider, schema, etc. in + // table properties, so that we can work around the Hive metastore issue about not case + // preserving and make Hive serde table support mixed-case column names. + val tableWithDataSourceProps = tableDefinition.copy( + properties = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition)) + client.createTable(tableWithDataSourceProps, ignoreIfExists) } else { + // To work around some hive metastore issues, e.g. not case-preserving, bad decimal type + // support, no column nullability, etc., we should do some extra works before saving table + // metadata into Hive metastore: + // 1. Put table metadata like provider, schema, etc. in table properties. + // 2. Check if this table is hive compatible. + // 2.1 If it's not hive compatible, set location URI, schema, partition columns and bucket + // spec to empty and save table metadata to Hive. + // 2.2 If it's hive compatible, set serde information in table metadata and try to save + // it to Hive. If it fails, treat it as not hive compatible and go back to 2.1 + val tableProperties = tableMetaToTableProps(tableDefinition) + // Ideally we should not create a managed table with location, but Hive serde table can // specify location for managed table. And in [[CreateDataSourceTableAsSelectCommand]] we have // to create the table directory and write out data before we create this table, to avoid @@ -209,139 +227,106 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } else { tableDefinition.storage.locationUri } - - if (tableDefinition.provider.get == DDLUtils.HIVE_PROVIDER) { - val tableWithDataSourceProps = tableDefinition.copy( - // We can't leave `locationUri` empty and count on Hive metastore to set a default table - // location, because Hive metastore is not case preserving and the table name is always - // lower cased when appear in the default table path, which is not expected. - storage = tableDefinition.storage.copy(locationUri = tableLocation), - // Here we follow data source tables and put table metadata like provider, schema, etc. in - // table properties, so that we can work around the Hive metastore issue about not case - // preserving and make Hive serde table support mixed-case column names. - properties = tableDefinition.properties ++ tableMetaToTableProps(tableDefinition)) - client.createTable(tableWithDataSourceProps, ignoreIfExists) - } else { - createDataSourceTable( - tableDefinition.withNewStorage(locationUri = tableLocation), - ignoreIfExists) + // Ideally we should also put `locationUri` in table properties like provider, schema, etc. + // However, in older version of Spark we already store table location in storage properties + // with key "path". Here we keep this behaviour for backward compatibility. + val storagePropsWithLocation = tableDefinition.storage.properties ++ + tableLocation.map("path" -> _) + + // converts the table metadata to Spark SQL specific format, i.e. set data schema, names and + // bucket specification to empty. Note that partition columns are retained, so that we can + // call partition-related Hive API later. + def newSparkSQLSpecificMetastoreTable(): CatalogTable = { + tableDefinition.copy( + // Hive only allows directory paths as location URIs while Spark SQL data source tables + // also allow file paths. For non-hive-compatible format, we should not set location URI + // to avoid hive metastore to throw exception. + storage = tableDefinition.storage.copy( + locationUri = None, + properties = storagePropsWithLocation), + schema = tableDefinition.partitionSchema, + bucketSpec = None, + properties = tableDefinition.properties ++ tableProperties) } - } - } - private def createDataSourceTable( - tableDefinition: CatalogTable, - ignoreIfExists: Boolean): Unit = { - // To work around some hive metastore issues, e.g. not case-preserving, bad decimal type - // support, no column nullability, etc., we should do some extra works before saving table - // metadata into Hive metastore: - // 1. Put table metadata like provider, schema, etc. in table properties. - // 2. Check if this table is hive compatible. - // 2.1 If it's not hive compatible, set location URI, schema, partition columns and bucket - // spec to empty and save table metadata to Hive. - // 2.2 If it's hive compatible, set serde information in table metadata and try to save - // it to Hive. If it fails, treat it as not hive compatible and go back to 2.1 - val tableProperties = tableMetaToTableProps(tableDefinition) - - // Ideally we should also put `locationUri` in table properties like provider, schema, etc. - // However, in older version of Spark we already store table location in storage properties - // with key "path". Here we keep this behaviour for backward compatibility. - val storagePropsWithLocation = tableDefinition.storage.properties ++ - tableDefinition.storage.locationUri.map("path" -> _) - - // converts the table metadata to Spark SQL specific format, i.e. set data schema, names and - // bucket specification to empty. Note that partition columns are retained, so that we can - // call partition-related Hive API later. - def newSparkSQLSpecificMetastoreTable(): CatalogTable = { - tableDefinition.copy( - // Hive only allows directory paths as location URIs while Spark SQL data source tables - // also allow file paths. For non-hive-compatible format, we should not set location URI - // to avoid hive metastore to throw exception. - storage = tableDefinition.storage.copy( - locationUri = None, - properties = storagePropsWithLocation), - schema = tableDefinition.partitionSchema, - bucketSpec = None, - properties = tableDefinition.properties ++ tableProperties) - } + // converts the table metadata to Hive compatible format, i.e. set the serde information. + def newHiveCompatibleMetastoreTable(serde: HiveSerDe): CatalogTable = { + val location = if (tableDefinition.tableType == EXTERNAL) { + // When we hit this branch, we are saving an external data source table with hive + // compatible format, which means the data source is file-based and must have a `path`. + require(tableDefinition.storage.locationUri.isDefined, + "External file-based data source table must have a `path` entry in storage properties.") + Some(new Path(tableDefinition.storage.locationUri.get).toUri.toString) + } else { + None + } - // converts the table metadata to Hive compatible format, i.e. set the serde information. - def newHiveCompatibleMetastoreTable(serde: HiveSerDe): CatalogTable = { - val location = if (tableDefinition.tableType == EXTERNAL) { - // When we hit this branch, we are saving an external data source table with hive - // compatible format, which means the data source is file-based and must have a `path`. - require(tableDefinition.storage.locationUri.isDefined, - "External file-based data source table must have a `path` entry in storage properties.") - Some(new Path(tableDefinition.location).toUri.toString) - } else { - None + tableDefinition.copy( + storage = tableDefinition.storage.copy( + locationUri = location, + inputFormat = serde.inputFormat, + outputFormat = serde.outputFormat, + serde = serde.serde, + properties = storagePropsWithLocation + ), + properties = tableDefinition.properties ++ tableProperties) } - tableDefinition.copy( - storage = tableDefinition.storage.copy( - locationUri = location, - inputFormat = serde.inputFormat, - outputFormat = serde.outputFormat, - serde = serde.serde, - properties = storagePropsWithLocation - ), - properties = tableDefinition.properties ++ tableProperties) - } - - val qualifiedTableName = tableDefinition.identifier.quotedString - val maybeSerde = HiveSerDe.sourceToSerDe(tableDefinition.provider.get) - val skipHiveMetadata = tableDefinition.storage.properties - .getOrElse("skipHiveMetadata", "false").toBoolean - - val (hiveCompatibleTable, logMessage) = maybeSerde match { - case _ if skipHiveMetadata => - val message = - s"Persisting data source table $qualifiedTableName into Hive metastore in" + - "Spark SQL specific format, which is NOT compatible with Hive." - (None, message) - - // our bucketing is un-compatible with hive(different hash function) - case _ if tableDefinition.bucketSpec.nonEmpty => - val message = - s"Persisting bucketed data source table $qualifiedTableName into " + - "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " - (None, message) - - case Some(serde) => - val message = - s"Persisting file based data source table $qualifiedTableName into " + - s"Hive metastore in Hive compatible format." - (Some(newHiveCompatibleMetastoreTable(serde)), message) - - case _ => - val provider = tableDefinition.provider.get - val message = - s"Couldn't find corresponding Hive SerDe for data source provider $provider. " + - s"Persisting data source table $qualifiedTableName into Hive metastore in " + - s"Spark SQL specific format, which is NOT compatible with Hive." - (None, message) - } + val qualifiedTableName = tableDefinition.identifier.quotedString + val maybeSerde = HiveSerDe.sourceToSerDe(tableDefinition.provider.get) + val skipHiveMetadata = tableDefinition.storage.properties + .getOrElse("skipHiveMetadata", "false").toBoolean + + val (hiveCompatibleTable, logMessage) = maybeSerde match { + case _ if skipHiveMetadata => + val message = + s"Persisting data source table $qualifiedTableName into Hive metastore in" + + "Spark SQL specific format, which is NOT compatible with Hive." + (None, message) + + // our bucketing is un-compatible with hive(different hash function) + case _ if tableDefinition.bucketSpec.nonEmpty => + val message = + s"Persisting bucketed data source table $qualifiedTableName into " + + "Hive metastore in Spark SQL specific format, which is NOT compatible with Hive. " + (None, message) + + case Some(serde) => + val message = + s"Persisting file based data source table $qualifiedTableName into " + + s"Hive metastore in Hive compatible format." + (Some(newHiveCompatibleMetastoreTable(serde)), message) + + case _ => + val provider = tableDefinition.provider.get + val message = + s"Couldn't find corresponding Hive SerDe for data source provider $provider. " + + s"Persisting data source table $qualifiedTableName into Hive metastore in " + + s"Spark SQL specific format, which is NOT compatible with Hive." + (None, message) + } - (hiveCompatibleTable, logMessage) match { - case (Some(table), message) => - // We first try to save the metadata of the table in a Hive compatible way. - // If Hive throws an error, we fall back to save its metadata in the Spark SQL - // specific way. - try { - logInfo(message) - saveTableIntoHive(table, ignoreIfExists) - } catch { - case NonFatal(e) => - val warningMessage = - s"Could not persist ${tableDefinition.identifier.quotedString} in a Hive " + - "compatible way. Persisting it into Hive metastore in Spark SQL specific format." - logWarning(warningMessage, e) - saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) - } + (hiveCompatibleTable, logMessage) match { + case (Some(table), message) => + // We first try to save the metadata of the table in a Hive compatible way. + // If Hive throws an error, we fall back to save its metadata in the Spark SQL + // specific way. + try { + logInfo(message) + saveTableIntoHive(table, ignoreIfExists) + } catch { + case NonFatal(e) => + val warningMessage = + s"Could not persist ${tableDefinition.identifier.quotedString} in a Hive " + + "compatible way. Persisting it into Hive metastore in Spark SQL specific format." + logWarning(warningMessage, e) + saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) + } - case (None, message) => - logWarning(message) - saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) + case (None, message) => + logWarning(message) + saveTableIntoHive(newSparkSQLSpecificMetastoreTable(), ignoreIfExists) + } } } From fae929e6ce9c6a0381e2922df2a4a5764429d966 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 8 Nov 2016 19:42:53 -0800 Subject: [PATCH 19/27] Tue Nov 8 19:42:53 PST 2016 --- .../internal/io/FileCommitProtocol.scala | 5 +- .../io/HadoopMapReduceCommitProtocol.scala | 5 + .../sql/catalyst/parser/AstBuilder.scala | 5 +- .../execution/datasources/DataSource.scala | 20 +- .../datasources/FileFormatWriter.scala | 3 +- .../ManifestFileCommitProtocol.scala | 6 + .../PartitionProviderCompatibilitySuite.scala | 234 +++++++++++------- 7 files changed, 171 insertions(+), 107 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala index 19746143b67e..f5549e342412 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -90,10 +90,7 @@ abstract class FileCommitProtocol { * Depending on the implementation, there may be weaker guarantees around adding files this way. */ def newTaskTempFileAbsPath( - taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { - throw new UnsupportedOperationException( - s"$this does not support adding files with an absolute path") - } + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String /** * Commits a task after the writes succeed. Must be called on the executors when running tasks. diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 6f8741d67662..b5962e84ca96 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -47,9 +47,14 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) /** * Tracks files staged by this task for absolute output paths. These outputs are not managed by * the Hadoop OutputCommitter, so we must move these to their final locations on job commit. + * + * The mapping is from the temp output path to the final desired output path of the file. */ @transient private var addedAbsPathFiles: mutable.Map[String, String] = null + /** + * The staging directory for all files committed with absolute output paths. + */ private def absPathStagingDir: Path = new Path(path, "_temporary-" + jobId) protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c912d79c7f55..2006844923cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -172,13 +172,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val tableIdent = visitTableIdentifier(ctx.tableIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) - val dynamicPartitionKeys = partitionKeys.filter(_._2.isEmpty) + val dynamicPartitionKeys: Map[String, Option[String]] = partitionKeys.filter(_._2.isEmpty) if (ctx.EXISTS != null && dynamicPartitionKeys.nonEmpty) { throw new ParseException(s"Dynamic partitions do not support IF NOT EXISTS. Specified " + "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx) } val overwrite = ctx.OVERWRITE != null - val staticPartitionKeys = partitionKeys.filter(_._2.nonEmpty).map(t => (t._1, t._2.get)) + val staticPartitionKeys: Map[String, String] = + partitionKeys.filter(_._2.nonEmpty).map(t => (t._1, t._2.get)) InsertIntoTable( UnresolvedRelation(tableIdent, None), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index a4c323595a0e..06b5f3167f33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -417,16 +417,16 @@ case class DataSource( // will be adjusted within InsertIntoHadoopFsRelation. val plan = InsertIntoHadoopFsRelationCommand( - outputPath, - Map.empty, - Map.empty, - columns, - bucketSpec, - format, - _ => Unit, // No existing table needs to be refreshed. - options, - data.logicalPlan, - mode) + outputPath = outputPath, + staticPartitionKeys = Map.empty, + customPartitionLocations = Map.empty, + partitionColumns = columns, + bucketSpec = bucketSpec, + fileFormat = format, + refreshFunction = _ => Unit, // No existing table needs to be refreshed. + options = options, + query = data.logicalPlan, + mode = mode) sparkSession.sessionState.executePlan(plan).toRdd // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring it. copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 2091465303e7..069142538c12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -321,7 +321,8 @@ object FileFormatWriter extends Logging { None } val path = if (customPath.isDefined) { - committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) + committer.newTaskTempFileAbsPath( + taskAttemptContext, customPath.get, java.util.UUID.randomUUID().toString + ext) } else { committer.newTaskTempFile(taskAttemptContext, partDir, ext) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index 1fe13fa1623f..92191c8b64b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -96,6 +96,12 @@ class ManifestFileCommitProtocol(jobId: String, path: String) file } + override def newTaskTempFileAbsPath( + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { + throw new UnsupportedOperationException( + s"$this does not support adding files with an absolute path") + } + override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { if (addedFiles.nonEmpty) { val fs = new Path(addedFiles.head).getFileSystem(taskContext.getConfiguration) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index 3c83736ddd3d..107ff25174e5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -24,10 +24,12 @@ import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils class PartitionProviderCompatibilitySuite extends QueryTest with TestHiveSingleton with SQLTestUtils { +/* private def setupPartitionedDatasourceTable(tableName: String, dir: File): Unit = { spark.range(5).selectExpr("id as fieldOne", "id as partCol").write .partitionBy("partCol") @@ -186,106 +188,158 @@ class PartitionProviderCompatibilitySuite } } } +*/ - test("insert into and overwrite new datasource tables with partial specs and custom locs") { - withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { - withTable("test") { - withTempDir { dir => - spark.range(5).selectExpr("id", "id as p1", "id as p2").write - .partitionBy("p1", "p2") - .mode("overwrite") - .parquet(dir.getAbsolutePath) - spark.sql(s""" - |create table test (id long, p1 int, p2 int) - |using parquet - |options (path "${dir.getAbsolutePath}") - |partitioned by (p1, p2)""".stripMargin) - spark.sql("msck repair table test") - assert(spark.sql("select * from test").count() == 5) - - // dynamic append with partial spec, existing dir - spark.sql("insert into test partition (p1=1, p2) select id, id from range(10)") - assert(spark.sql("select * from test where p1=1").count() == 11) - assert(spark.sql("select * from test where p1=1 and p2=1").count() == 2) - - // dynamic append with full spec, existing dir - spark.sql("insert into test partition (p1=1, p2=1) select id from range(10)") - assert(spark.sql("select * from test where p1=1").count() == 21) - assert(spark.sql("select * from test where p1=1 and p2=1").count() == 12) - - // dynamic append with partial spec, new dir - spark.sql("insert into test partition (p1=100, p2) select id, id from range(10)") - assert(spark.sql("select * from test where p1=100").count() == 10) + /** + * Runs a test against a multi-level partitioned table. + * + * The initial partitioning structure is: + * /p1=0/p2=0 -- custom location a + * /p1=0/p2=1 -- custom location b + * /p1=1/p2=0 -- custom location c + * /p1=1/p2=1 -- default location + */ + private def testCustomLocations(testFn: => Unit): Unit = { + val base = Utils.createTempDir(namePrefix = "base") + val a = Utils.createTempDir(namePrefix = "a") + val b = Utils.createTempDir(namePrefix = "b") + val c = Utils.createTempDir(namePrefix = "c") + try { + spark.sql(s""" + |create table test (id long, p1 int, p2 int) + |using parquet + |options (path "${base.getAbsolutePath}") + |partitioned by (p1, p2)""".stripMargin) + spark.sql(s"alter table test add partition (p1=0, p2=0) location '${a.getAbsolutePath}'") + spark.sql(s"alter table test add partition (p1=0, p2=1) location '${b.getAbsolutePath}'") + spark.sql(s"alter table test add partition (p1=1, p2=0) location '${c.getAbsolutePath}'") + spark.sql(s"alter table test add partition (p1=1, p2=1)") - // dynamic append with full spec, new dir - spark.sql("insert into test partition (p1=100, p2=100) select id from range(10)") - assert(spark.sql("select * from test where p1=100").count() == 20) - assert(spark.sql("show partitions test").count() == 25) + testFn - // dynamic overwrite with partial spec, existing dir - spark.sql( - "insert overwrite table test partition (p1=1, p2) select id, id from range(100)") - assert(spark.sql("select * from test where p1=1").count() == 100) - assert(spark.sql("show partitions test").count() == 115) + // Now validate the partition custom locations were respected + val initialCount = spark.sql("select * from test").count() + val numA = spark.sql("select * from test where p1=0 and p2=0").count() + val numB = spark.sql("select * from test where p1=0 and p2=1").count() + val numC = spark.sql("select * from test where p1=1 and p2=0").count() + Utils.deleteRecursively(a) + spark.sql("refresh table test") + assert(spark.sql("select * from test where p1=0 and p2=0").count() == 0) + assert(spark.sql("select * from test").count() == initialCount - numA) + Utils.deleteRecursively(b) + spark.sql("refresh table test") + assert(spark.sql("select * from test where p1=0 and p2=1").count() == 0) + assert(spark.sql("select * from test").count() == initialCount - numA - numB) + Utils.deleteRecursively(c) + spark.sql("refresh table test") + assert(spark.sql("select * from test where p1=1 and p2=0").count() == 0) + assert(spark.sql("select * from test").count() == initialCount - numA - numB - numC) + } finally { + Utils.deleteRecursively(base) + Utils.deleteRecursively(a) + Utils.deleteRecursively(b) + Utils.deleteRecursively(c) + spark.sql("drop table test") + } + } - // dynamic overwrite with full spec, existing dir - spark.sql( - "insert overwrite table test partition (p1=1, p2=1) select id from range(100)") - assert(spark.sql("select * from test where p1=1").count() == 199) - assert(spark.sql("select * from test where p1=1 and p2=1").count() == 100) - assert(spark.sql("show partitions test").count() == 115) + test("sanity check table setup") { + testCustomLocations { + assert(spark.sql("select * from test").count() == 0) + assert(spark.sql("show partitions test").count() == 4) + } + } - // dynamic overwrite with partial spec, new dir - spark.sql( - "insert overwrite table test partition (p1=500, p2) select id, id from range(10)") - assert(spark.sql("select * from test where p1=500").count() == 10) - assert(spark.sql("show partitions test").count() == 125) + test("insert into partial dynamic partitions") { + testCustomLocations { + spark.sql("insert into test partition (p1=0, p2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 12) + spark.sql("insert into test partition (p1=0, p2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 20) + assert(spark.sql("show partitions test").count() == 12) + spark.sql("insert into test partition (p1=1, p2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 30) + assert(spark.sql("show partitions test").count() == 20) + spark.sql("insert into test partition (p1=2, p2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 40) + assert(spark.sql("show partitions test").count() == 30) + } + } - // dynamic overwrite with partial spec again (test partition cleanup) - spark.sql( - "insert overwrite table test partition (p1=1, p2) select id, id from range(10)") - assert(spark.sql("select * from test where p1=1").count() == 10) - assert(spark.sql("show partitions test").count() == 35) + test("insert into fully dynamic partitions") { + testCustomLocations { + spark.sql("insert into test partition (p1, p2) select id, id, id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 12) + spark.sql("insert into test partition (p1, p2) select id, id, id from range(10)") + assert(spark.sql("select * from test").count() == 20) + assert(spark.sql("show partitions test").count() == 12) + } + } - // dynamic overwrite with full spec, new dir - spark.sql( - "insert overwrite table test partition (p1=500, p2=500) select id from range(10)") - assert(spark.sql("select * from test where p1=500 and p2=500").count() == 10) + test("insert into static partition") { + testCustomLocations { + spark.sql("insert into test partition (p1=0, p2=0) select id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert into test partition (p1=0, p2=0) select id from range(10)") + assert(spark.sql("select * from test").count() == 20) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert into test partition (p1=1, p2=1) select id from range(10)") + assert(spark.sql("select * from test").count() == 30) + assert(spark.sql("show partitions test").count() == 4) + } + } - // overwrite entire table - spark.sql("insert overwrite table test select id, 1, 1 from range(10)") - assert(spark.sql("select * from test").count() == 10) - assert(spark.sql("show partitions test").count() == 1) + test("overwrite partial dynamic partitions") { + testCustomLocations { + spark.sql("insert overwrite table test partition (p1=0, p2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 12) + spark.sql("insert overwrite table test partition (p1=0, p2) select id, id from range(5)") + assert(spark.sql("select * from test").count() == 5) + assert(spark.sql("show partitions test").count() == 7) + spark.sql("insert overwrite table test partition (p1=0, p2) select id, id from range(1)") + assert(spark.sql("select * from test").count() == 1) + assert(spark.sql("show partitions test").count() == 3) + spark.sql("insert overwrite table test partition (p1=1, p2) select id, id from range(10)") + assert(spark.sql("select * from test").count() == 11) + assert(spark.sql("show partitions test").count() == 11) + spark.sql("insert overwrite table test partition (p1=1, p2) select id, id from range(1)") + assert(spark.sql("select * from test").count() == 2) + assert(spark.sql("show partitions test").count() == 2) + spark.sql("insert overwrite table test partition (p1=3, p2) select id, id from range(100)") + assert(spark.sql("select * from test").count() == 102) + assert(spark.sql("show partitions test").count() == 102) + } + } - // dynamic append to custom location - withTempDir { a => - spark.sql( - s"alter table test add partition (p1=1, p2=2) location '${a.getAbsolutePath}'" - ).count() - spark.sql("insert into test partition (p1=1, p2) select id, id from range(100)") - spark.sql("insert into test partition (p1=1, p2) select id, id from range(100)") - assert(spark.sql("select * from test where p1=1").count() == 210) - assert(spark.sql("select * from test where p1=1 and p2=2").count() == 2) - } - sql("refresh table test") - assert(spark.sql("select * from test where p1=1 and p2=2").count() == 0) + test("overwrite fully dynamic partitions") { + testCustomLocations { + spark.sql("insert overwrite table test partition (p1, p2) select id, id, id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 10) + spark.sql("insert overwrite table test partition (p1, p2) select id, id, id from range(5)") + assert(spark.sql("select * from test").count() == 5) + assert(spark.sql("show partitions test").count() == 5) + } + } - // dynamic overwrite of custom locations - withTempDir { a => - spark.sql( - s"alter table test partition (p1=1, p2=2) set location '${a.getAbsolutePath}'" - ).count() - spark.sql( - "insert overwrite table test partition (p1=1, p2) select id, id from range(100)") - spark.sql( - "insert overwrite table test partition (p1=1, p2) select id, id from range(100)") - assert(spark.sql("select * from test where p1=1").count() == 100) - assert(spark.sql("select * from test where p1=1 and p2=2").count() == 1) - } - sql("refresh table test") - assert(spark.sql("select * from test where p1=1 and p2=2").count() == 0) - } - } + test("overwrite static partition") { + testCustomLocations { + spark.sql("insert overwrite table test partition (p1=0, p2=0) select id from range(10)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert overwrite table test partition (p1=0, p2=0) select id from range(5)") + assert(spark.sql("select * from test").count() == 5) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert overwrite table test partition (p1=1, p2=1) select id from range(5)") + assert(spark.sql("select * from test").count() == 10) + assert(spark.sql("show partitions test").count() == 4) + spark.sql("insert overwrite table test partition (p1=1, p2=2) select id from range(5)") + assert(spark.sql("select * from test").count() == 15) + assert(spark.sql("show partitions test").count() == 5) } } } From fbd7b425290644fcfdcaf92491aaae2ac67eefb9 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 8 Nov 2016 19:43:30 -0800 Subject: [PATCH 20/27] Tue Nov 8 19:43:30 PST 2016 --- .../spark/sql/hive/PartitionProviderCompatibilitySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index 107ff25174e5..6237ddd9e24a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -191,7 +191,7 @@ class PartitionProviderCompatibilitySuite */ /** - * Runs a test against a multi-level partitioned table. + * Runs a test against a multi-level partitioned table, and validate some post-test invariants. * * The initial partitioning structure is: * /p1=0/p2=0 -- custom location a From 4296612bd373bc1118b2cbfc10530b1895aaef60 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 8 Nov 2016 20:04:12 -0800 Subject: [PATCH 21/27] Tue Nov 8 20:04:12 PST 2016 --- .../org/apache/spark/internal/io/FileCommitProtocol.scala | 6 ++++++ .../spark/sql/execution/datasources/FileFormatWriter.scala | 4 +++- .../sql/hive/PartitionProviderCompatibilitySuite.scala | 3 ++- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala index f5549e342412..1c935c38fdb9 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -82,12 +82,18 @@ abstract class FileCommitProtocol { * * The "dir" parameter specifies 2, and "ext" parameter specifies both 4 and 5, and the rest * are left to the commit protocol implementation to decide. + * + * Important: it is the caller's responsibility to add uniquely identifying content to "ext" + * if a task is going to write out multiple files to the same dir. */ def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String /** * Similar to newTaskTempFile(), but allows files to committed to an absolute output location. * Depending on the implementation, there may be weaker guarantees around adding files this way. + * + * Important: it is the caller's responsibility to add uniquely identifying content to "ext" + * if a task is going to write out multiple files at all (even to different directories). */ def newTaskTempFileAbsPath( taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 069142538c12..d80e3273ef95 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -321,8 +321,10 @@ object FileFormatWriter extends Logging { None } val path = if (customPath.isDefined) { + // We need to include a uuid here since the commit protocol does not guarantee that + // temp files requested by the same task for absolute placement do not collide. committer.newTaskTempFileAbsPath( - taskAttemptContext, customPath.get, java.util.UUID.randomUUID().toString + ext) + taskAttemptContext, customPath.get, UUID.randomUUID().toString + ext) } else { committer.newTaskTempFile(taskAttemptContext, partDir, ext) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index 6237ddd9e24a..bd28f7ccfdbf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -191,7 +191,8 @@ class PartitionProviderCompatibilitySuite */ /** - * Runs a test against a multi-level partitioned table, and validate some post-test invariants. + * Runs a test against a multi-level partitioned table, then validates that the custom locations + * were respected by the output writer. * * The initial partitioning structure is: * /p1=0/p2=0 -- custom location a From 001cb1d1133a1a987b31431f6932a1fa0692f404 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 8 Nov 2016 20:09:32 -0800 Subject: [PATCH 22/27] Tue Nov 8 20:09:31 PST 2016 --- .../apache/spark/internal/io/FileCommitProtocol.scala | 6 ++++-- .../internal/io/HadoopMapReduceCommitProtocol.scala | 9 +++++++-- .../sql/execution/datasources/FileFormatWriter.scala | 5 +---- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala index 1c935c38fdb9..afd2250c93a8 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -84,7 +84,8 @@ abstract class FileCommitProtocol { * are left to the commit protocol implementation to decide. * * Important: it is the caller's responsibility to add uniquely identifying content to "ext" - * if a task is going to write out multiple files to the same dir. + * if a task is going to write out multiple files to the same dir. The file commit protocol only + * guarantees that files written by different tasks will not conflict. */ def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String @@ -93,7 +94,8 @@ abstract class FileCommitProtocol { * Depending on the implementation, there may be weaker guarantees around adding files this way. * * Important: it is the caller's responsibility to add uniquely identifying content to "ext" - * if a task is going to write out multiple files at all (even to different directories). + * if a task is going to write out multiple files to the same dir. The file commit protocol only + * guarantees that files written by different tasks will not conflict. */ def newTaskTempFileAbsPath( taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index b5962e84ca96..d9aea09863f0 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -17,7 +17,7 @@ package org.apache.spark.internal.io -import java.util.Date +import java.util.{Date, UUID} import scala.collection.mutable @@ -82,7 +82,12 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { val filename = getFilename(taskContext, ext) val absOutputPath = new Path(absoluteDir, filename).toString - val tmpOutputPath = new Path(absPathStagingDir, filename).toString + + // Include a UUID here to prevent file collisions for one task writing to different dirs. + // In principle we could include hash(absoluteDir) instead but this is simpler. + val tmpOutputPath = new Path( + absPathStagingDir, UUID.randomUUID().toString() + "-" + filename).toString + addedAbsPathFiles(tmpOutputPath) = absOutputPath tmpOutputPath } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index d80e3273ef95..2091465303e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -321,10 +321,7 @@ object FileFormatWriter extends Logging { None } val path = if (customPath.isDefined) { - // We need to include a uuid here since the commit protocol does not guarantee that - // temp files requested by the same task for absolute placement do not collide. - committer.newTaskTempFileAbsPath( - taskAttemptContext, customPath.get, UUID.randomUUID().toString + ext) + committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) } else { committer.newTaskTempFile(taskAttemptContext, partDir, ext) } From 91f87de270bbd743f0b40b82016391706a936ca1 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 8 Nov 2016 20:11:02 -0800 Subject: [PATCH 23/27] Tue Nov 8 20:11:02 PST 2016 --- .../spark/sql/hive/PartitionProviderCompatibilitySuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index bd28f7ccfdbf..96e1790cc400 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -29,7 +29,6 @@ import org.apache.spark.util.Utils class PartitionProviderCompatibilitySuite extends QueryTest with TestHiveSingleton with SQLTestUtils { -/* private def setupPartitionedDatasourceTable(tableName: String, dir: File): Unit = { spark.range(5).selectExpr("id as fieldOne", "id as partCol").write .partitionBy("partCol") @@ -188,7 +187,6 @@ class PartitionProviderCompatibilitySuite } } } -*/ /** * Runs a test against a multi-level partitioned table, then validates that the custom locations From 9dbc3f12f85c376c813d0eb2ea12854e9afe3853 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 9 Nov 2016 13:34:17 +0800 Subject: [PATCH 24/27] fix test --- .../catalog/SessionCatalogSuite.scala | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index b77fef225a0c..6fef7750b0fb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -527,13 +527,13 @@ class SessionCatalogSuite extends SparkFunSuite { sessionCatalog.createTable(newTable("tbl", "mydb"), ignoreIfExists = false) sessionCatalog.createPartitions( TableIdentifier("tbl", Some("mydb")), Seq(part1, part2), ignoreIfExists = false) - assert(catalogPartitionsEqual(externalCatalog, "mydb", "tbl", Seq(part1, part2))) + assert(catalogPartitionsEqual(externalCatalog.listPartitions("mydb", "tbl"), part1, part2)) // Create partitions without explicitly specifying database sessionCatalog.setCurrentDatabase("mydb") sessionCatalog.createPartitions( TableIdentifier("tbl"), Seq(partWithMixedOrder), ignoreIfExists = false) assert(catalogPartitionsEqual( - externalCatalog, "mydb", "tbl", Seq(part1, part2, partWithMixedOrder))) + externalCatalog.listPartitions("mydb", "tbl"), part1, part2, partWithMixedOrder)) } test("create partitions when database/table does not exist") { @@ -586,13 +586,13 @@ class SessionCatalogSuite extends SparkFunSuite { test("drop partitions") { val externalCatalog = newBasicCatalog() val sessionCatalog = new SessionCatalog(externalCatalog) - assert(catalogPartitionsEqual(externalCatalog, "db2", "tbl2", Seq(part1, part2))) + assert(catalogPartitionsEqual(externalCatalog.listPartitions("db2", "tbl2"), part1, part2)) sessionCatalog.dropPartitions( TableIdentifier("tbl2", Some("db2")), Seq(part1.spec), ignoreIfNotExists = false, purge = false) - assert(catalogPartitionsEqual(externalCatalog, "db2", "tbl2", Seq(part2))) + assert(catalogPartitionsEqual(externalCatalog.listPartitions("db2", "tbl2"), part2)) // Drop partitions without explicitly specifying database sessionCatalog.setCurrentDatabase("db2") sessionCatalog.dropPartitions( @@ -604,7 +604,7 @@ class SessionCatalogSuite extends SparkFunSuite { // Drop multiple partitions at once sessionCatalog.createPartitions( TableIdentifier("tbl2", Some("db2")), Seq(part1, part2), ignoreIfExists = false) - assert(catalogPartitionsEqual(externalCatalog, "db2", "tbl2", Seq(part1, part2))) + assert(catalogPartitionsEqual(externalCatalog.listPartitions("db2", "tbl2"), part1, part2)) sessionCatalog.dropPartitions( TableIdentifier("tbl2", Some("db2")), Seq(part1.spec, part2.spec), @@ -844,10 +844,11 @@ class SessionCatalogSuite extends SparkFunSuite { test("list partitions") { val catalog = new SessionCatalog(newBasicCatalog()) - assert(catalog.listPartitions(TableIdentifier("tbl2", Some("db2"))).toSet == Set(part1, part2)) + assert(catalogPartitionsEqual( + catalog.listPartitions(TableIdentifier("tbl2", Some("db2"))), part1, part2)) // List partitions without explicitly specifying database catalog.setCurrentDatabase("db2") - assert(catalog.listPartitions(TableIdentifier("tbl2")).toSet == Set(part1, part2)) + assert(catalogPartitionsEqual(catalog.listPartitions(TableIdentifier("tbl2")), part1, part2)) } test("list partitions when database/table does not exist") { @@ -860,6 +861,15 @@ class SessionCatalogSuite extends SparkFunSuite { } } + private def catalogPartitionsEqual( + actualParts: Seq[CatalogTablePartition], + expectedParts: CatalogTablePartition*): Boolean = { + // ExternalCatalog may set a default location for partitions, here we ignore the partition + // location when comparing them. + actualParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet == + expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet + } + // -------------------------------------------------------------------------- // Functions // -------------------------------------------------------------------------- From 8e10ff73c8b3276696d1abaf4816db891a824e4b Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 9 Nov 2016 14:20:08 -0800 Subject: [PATCH 25/27] upper case the test --- .../PartitionProviderCompatibilitySuite.scala | 74 +++++++++---------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index 96e1790cc400..a1aa07456fd3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -193,10 +193,10 @@ class PartitionProviderCompatibilitySuite * were respected by the output writer. * * The initial partitioning structure is: - * /p1=0/p2=0 -- custom location a - * /p1=0/p2=1 -- custom location b - * /p1=1/p2=0 -- custom location c - * /p1=1/p2=1 -- default location + * /P1=0/P2=0 -- custom location a + * /P1=0/P2=1 -- custom location b + * /P1=1/P2=0 -- custom location c + * /P1=1/P2=1 -- default location */ private def testCustomLocations(testFn: => Unit): Unit = { val base = Utils.createTempDir(namePrefix = "base") @@ -205,33 +205,33 @@ class PartitionProviderCompatibilitySuite val c = Utils.createTempDir(namePrefix = "c") try { spark.sql(s""" - |create table test (id long, p1 int, p2 int) + |create table test (id long, P1 int, P2 int) |using parquet |options (path "${base.getAbsolutePath}") - |partitioned by (p1, p2)""".stripMargin) - spark.sql(s"alter table test add partition (p1=0, p2=0) location '${a.getAbsolutePath}'") - spark.sql(s"alter table test add partition (p1=0, p2=1) location '${b.getAbsolutePath}'") - spark.sql(s"alter table test add partition (p1=1, p2=0) location '${c.getAbsolutePath}'") - spark.sql(s"alter table test add partition (p1=1, p2=1)") + |partitioned by (P1, P2)""".stripMargin) + spark.sql(s"alter table test add partition (P1=0, P2=0) location '${a.getAbsolutePath}'") + spark.sql(s"alter table test add partition (P1=0, P2=1) location '${b.getAbsolutePath}'") + spark.sql(s"alter table test add partition (P1=1, P2=0) location '${c.getAbsolutePath}'") + spark.sql(s"alter table test add partition (P1=1, P2=1)") testFn // Now validate the partition custom locations were respected val initialCount = spark.sql("select * from test").count() - val numA = spark.sql("select * from test where p1=0 and p2=0").count() - val numB = spark.sql("select * from test where p1=0 and p2=1").count() - val numC = spark.sql("select * from test where p1=1 and p2=0").count() + val numA = spark.sql("select * from test where P1=0 and P2=0").count() + val numB = spark.sql("select * from test where P1=0 and P2=1").count() + val numC = spark.sql("select * from test where P1=1 and P2=0").count() Utils.deleteRecursively(a) spark.sql("refresh table test") - assert(spark.sql("select * from test where p1=0 and p2=0").count() == 0) + assert(spark.sql("select * from test where P1=0 and P2=0").count() == 0) assert(spark.sql("select * from test").count() == initialCount - numA) Utils.deleteRecursively(b) spark.sql("refresh table test") - assert(spark.sql("select * from test where p1=0 and p2=1").count() == 0) + assert(spark.sql("select * from test where P1=0 and P2=1").count() == 0) assert(spark.sql("select * from test").count() == initialCount - numA - numB) Utils.deleteRecursively(c) spark.sql("refresh table test") - assert(spark.sql("select * from test where p1=1 and p2=0").count() == 0) + assert(spark.sql("select * from test where P1=1 and P2=0").count() == 0) assert(spark.sql("select * from test").count() == initialCount - numA - numB - numC) } finally { Utils.deleteRecursively(base) @@ -251,16 +251,16 @@ class PartitionProviderCompatibilitySuite test("insert into partial dynamic partitions") { testCustomLocations { - spark.sql("insert into test partition (p1=0, p2) select id, id from range(10)") + spark.sql("insert into test partition (P1=0, P2) select id, id from range(10)") assert(spark.sql("select * from test").count() == 10) assert(spark.sql("show partitions test").count() == 12) - spark.sql("insert into test partition (p1=0, p2) select id, id from range(10)") + spark.sql("insert into test partition (P1=0, P2) select id, id from range(10)") assert(spark.sql("select * from test").count() == 20) assert(spark.sql("show partitions test").count() == 12) - spark.sql("insert into test partition (p1=1, p2) select id, id from range(10)") + spark.sql("insert into test partition (P1=1, P2) select id, id from range(10)") assert(spark.sql("select * from test").count() == 30) assert(spark.sql("show partitions test").count() == 20) - spark.sql("insert into test partition (p1=2, p2) select id, id from range(10)") + spark.sql("insert into test partition (P1=2, P2) select id, id from range(10)") assert(spark.sql("select * from test").count() == 40) assert(spark.sql("show partitions test").count() == 30) } @@ -268,10 +268,10 @@ class PartitionProviderCompatibilitySuite test("insert into fully dynamic partitions") { testCustomLocations { - spark.sql("insert into test partition (p1, p2) select id, id, id from range(10)") + spark.sql("insert into test partition (P1, P2) select id, id, id from range(10)") assert(spark.sql("select * from test").count() == 10) assert(spark.sql("show partitions test").count() == 12) - spark.sql("insert into test partition (p1, p2) select id, id, id from range(10)") + spark.sql("insert into test partition (P1, P2) select id, id, id from range(10)") assert(spark.sql("select * from test").count() == 20) assert(spark.sql("show partitions test").count() == 12) } @@ -279,13 +279,13 @@ class PartitionProviderCompatibilitySuite test("insert into static partition") { testCustomLocations { - spark.sql("insert into test partition (p1=0, p2=0) select id from range(10)") + spark.sql("insert into test partition (P1=0, P2=0) select id from range(10)") assert(spark.sql("select * from test").count() == 10) assert(spark.sql("show partitions test").count() == 4) - spark.sql("insert into test partition (p1=0, p2=0) select id from range(10)") + spark.sql("insert into test partition (P1=0, P2=0) select id from range(10)") assert(spark.sql("select * from test").count() == 20) assert(spark.sql("show partitions test").count() == 4) - spark.sql("insert into test partition (p1=1, p2=1) select id from range(10)") + spark.sql("insert into test partition (P1=1, P2=1) select id from range(10)") assert(spark.sql("select * from test").count() == 30) assert(spark.sql("show partitions test").count() == 4) } @@ -293,22 +293,22 @@ class PartitionProviderCompatibilitySuite test("overwrite partial dynamic partitions") { testCustomLocations { - spark.sql("insert overwrite table test partition (p1=0, p2) select id, id from range(10)") + spark.sql("insert overwrite table test partition (P1=0, P2) select id, id from range(10)") assert(spark.sql("select * from test").count() == 10) assert(spark.sql("show partitions test").count() == 12) - spark.sql("insert overwrite table test partition (p1=0, p2) select id, id from range(5)") + spark.sql("insert overwrite table test partition (P1=0, P2) select id, id from range(5)") assert(spark.sql("select * from test").count() == 5) assert(spark.sql("show partitions test").count() == 7) - spark.sql("insert overwrite table test partition (p1=0, p2) select id, id from range(1)") + spark.sql("insert overwrite table test partition (P1=0, P2) select id, id from range(1)") assert(spark.sql("select * from test").count() == 1) assert(spark.sql("show partitions test").count() == 3) - spark.sql("insert overwrite table test partition (p1=1, p2) select id, id from range(10)") + spark.sql("insert overwrite table test partition (P1=1, P2) select id, id from range(10)") assert(spark.sql("select * from test").count() == 11) assert(spark.sql("show partitions test").count() == 11) - spark.sql("insert overwrite table test partition (p1=1, p2) select id, id from range(1)") + spark.sql("insert overwrite table test partition (P1=1, P2) select id, id from range(1)") assert(spark.sql("select * from test").count() == 2) assert(spark.sql("show partitions test").count() == 2) - spark.sql("insert overwrite table test partition (p1=3, p2) select id, id from range(100)") + spark.sql("insert overwrite table test partition (P1=3, P2) select id, id from range(100)") assert(spark.sql("select * from test").count() == 102) assert(spark.sql("show partitions test").count() == 102) } @@ -316,10 +316,10 @@ class PartitionProviderCompatibilitySuite test("overwrite fully dynamic partitions") { testCustomLocations { - spark.sql("insert overwrite table test partition (p1, p2) select id, id, id from range(10)") + spark.sql("insert overwrite table test partition (P1, P2) select id, id, id from range(10)") assert(spark.sql("select * from test").count() == 10) assert(spark.sql("show partitions test").count() == 10) - spark.sql("insert overwrite table test partition (p1, p2) select id, id, id from range(5)") + spark.sql("insert overwrite table test partition (P1, P2) select id, id, id from range(5)") assert(spark.sql("select * from test").count() == 5) assert(spark.sql("show partitions test").count() == 5) } @@ -327,16 +327,16 @@ class PartitionProviderCompatibilitySuite test("overwrite static partition") { testCustomLocations { - spark.sql("insert overwrite table test partition (p1=0, p2=0) select id from range(10)") + spark.sql("insert overwrite table test partition (P1=0, P2=0) select id from range(10)") assert(spark.sql("select * from test").count() == 10) assert(spark.sql("show partitions test").count() == 4) - spark.sql("insert overwrite table test partition (p1=0, p2=0) select id from range(5)") + spark.sql("insert overwrite table test partition (P1=0, P2=0) select id from range(5)") assert(spark.sql("select * from test").count() == 5) assert(spark.sql("show partitions test").count() == 4) - spark.sql("insert overwrite table test partition (p1=1, p2=1) select id from range(5)") + spark.sql("insert overwrite table test partition (P1=1, P2=1) select id from range(5)") assert(spark.sql("select * from test").count() == 10) assert(spark.sql("show partitions test").count() == 4) - spark.sql("insert overwrite table test partition (p1=1, p2=2) select id from range(5)") + spark.sql("insert overwrite table test partition (P1=1, P2=2) select id from range(5)") assert(spark.sql("select * from test").count() == 15) assert(spark.sql("show partitions test").count() == 5) } From 15005661d806f717cf13998fd23abf3d1e06a4d3 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 9 Nov 2016 14:23:33 -0800 Subject: [PATCH 26/27] merge case sensitivity fixes --- .../datasources/InsertIntoHadoopFsRelationCommand.scala | 6 +++--- .../spark/sql/execution/datasources/PartitioningUtils.scala | 1 + 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 69642ed4c4a5..28975e1546e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -56,6 +56,8 @@ case class InsertIntoHadoopFsRelationCommand( catalogTable: Option[CatalogTable]) extends RunnableCommand { + import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName + override protected def innerChildren: Seq[LogicalPlan] = query :: Nil override def run(sparkSession: SparkSession): Seq[Row] = { @@ -124,9 +126,7 @@ case class InsertIntoHadoopFsRelationCommand( "/" + partitionColumns.flatMap { p => staticPartitionKeys.get(p.name) match { case Some(value) => - Some( - PartitioningUtils.escapePathName(p.name) + "=" + - PartitioningUtils.escapePathName(value)) + Some(escapePathName(p.name) + "=" + escapePathName(value)) case None => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 907e73da9733..409b85b1fe6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -62,6 +62,7 @@ object PartitioningUtils { } import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.DEFAULT_PARTITION_NAME + import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName /** From 51c132257cd5ef45a00d00b391d2ad4ecbe108d7 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 9 Nov 2016 16:21:52 -0800 Subject: [PATCH 27/27] fix crash in commit protocol --- .../spark/internal/io/HadoopMapReduceCommitProtocol.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 58c580445613..b2d9b8d2a012 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -126,7 +126,8 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String) override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { committer.commitJob(jobContext) - val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]]).reduce(_ ++ _) + val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]]) + .foldLeft(Map[String, String]())(_ ++ _) logDebug(s"Committing files staged for absolute locations $filesToMove") val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration) for ((src, dst) <- filesToMove) {