From 6ca7771a39fe7c6ca5f82acc7d76dff00238de68 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 16 Aug 2017 18:32:23 -0700 Subject: [PATCH 01/47] add insert overwrite local directory --- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 + .../UnsupportedOperationChecker.scala | 4 + .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 14 +- .../spark/sql/execution/SparkSqlParser.scala | 35 ++++- .../spark/sql/hive/HiveStrategies.scala | 6 +- .../hive/execution/InsertIntoDirCommand.scala | 128 ++++++++++++++++++ .../hive/execution/InsertIntoHiveTable.scala | 41 +----- .../sql/hive/execution/SaveAsHiveFile.scala | 75 ++++++++++ 9 files changed, 267 insertions(+), 41 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 954955b6b129..3c1753ab81b4 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -240,6 +240,7 @@ query insertInto : INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)? | INSERT INTO TABLE? tableIdentifier partitionSpec? + | INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? ; partitionSpecLocation @@ -740,6 +741,7 @@ nonReserved | AND | CASE | CAST | DISTINCT | DIV | ELSE | END | FUNCTION | INTERVAL | MACRO | OR | STRATIFY | THEN | UNBOUNDED | WHEN | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT | CURRENT_DATE | CURRENT_TIMESTAMP + | DIRECTORY ; SELECT: 'SELECT'; @@ -810,6 +812,7 @@ WITH: 'WITH'; VALUES: 'VALUES'; CREATE: 'CREATE'; TABLE: 'TABLE'; +DIRECTORY: 'DIRECTORY'; VIEW: 'VIEW'; REPLACE: 'REPLACE'; INSERT: 'INSERT'; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 6ab4153bac70..f8a8f6c7fe4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -142,10 +142,14 @@ object UnsupportedOperationChecker { "Distinct aggregations are not supported on streaming DataFrames/Datasets. Consider " + "using approx_count_distinct() instead.") + case _: Command => throwError("Commands like CreateTable*, AlterTable*, Show* are not supported with " + "streaming DataFrames/Datasets") + case _: InsertIntoDir => + throwError("InsertIntoDir is not supported with streaming DataFrames/Datasets") + // mapGroupsWithState and flatMapGroupsWithState case m: FlatMapGroupsWithState if m.isStreaming => 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 22c5484b7663..efe7019ffcc0 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 @@ -176,7 +176,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging /** * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan. */ - private def withInsertInto( + protected def withInsertInto( ctx: InsertIntoContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { val tableIdent = visitTableIdentifier(ctx.tableIdentifier) 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 303014e0b8d3..f8543fc83cc1 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ @@ -359,6 +359,18 @@ case class InsertIntoTable( override lazy val resolved: Boolean = false } +case class InsertIntoDir( + path: String, + isLocal: Boolean, + rowStorage: CatalogStorageFormat, + fileStorage: CatalogStorageFormat, + child: LogicalPlan) + extends LogicalPlan { + + override def children: Seq[LogicalPlan] = child :: Nil + override def output: Seq[Attribute] = Seq.empty +} + /** * A container for holding the view description(CatalogTable), and the output of the view. The * child should be a logical plan parsed from the `CatalogTable.viewText`, should throw an error diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index d4414b6f78ca..7e007920cf51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -20,17 +20,16 @@ package org.apache.spark.sql.execution import java.util.Locale import scala.collection.JavaConverters._ - import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.TerminalNode - import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ -import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, _} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, _} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} @@ -1499,4 +1498,34 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { query: LogicalPlan): LogicalPlan = { RepartitionByExpression(expressions, query, conf.numShufflePartitions) } + + /** + * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE or INSERT INTO [LOCAL] DIRECOTRY + * operation to the logical plan. + */ + protected override def withInsertInto(ctx: InsertIntoContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + val tableIdent = Option(ctx.tableIdentifier).map(visitTableIdentifier) + val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) + + val dynamicPartitionKeys = 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) + } + + validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) + val rowStorage = Option(ctx.rowFormat).map(visitRowFormat) + .getOrElse(CatalogStorageFormat.empty) + val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) + .getOrElse(CatalogStorageFormat.empty) + + tableIdent match { + case Some(ti: TableIdentifier) => + InsertIntoTable(UnresolvedRelation(ti), partitionKeys, query, + ctx.OVERWRITE != null, ctx.EXISTS != null) + case _ => + InsertIntoDir(string(ctx.path), ctx.LOCAL != null, rowStorage, fileStorage, query) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index ae1e7e72e8c3..19b2a7f16837 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -21,12 +21,11 @@ import java.io.IOException import java.util.Locale import org.apache.hadoop.fs.{FileSystem, Path} - import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, ScriptTransformation} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, ScriptTransformation} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} @@ -155,6 +154,9 @@ object HiveAnalysis extends Rule[LogicalPlan] { case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => CreateHiveTableAsSelectCommand(tableDesc, query, mode) + + case InsertIntoDir(path, isLocal, rowStorage, fileStorage, child) => + InsertIntoDirCommand(path, isLocal, rowStorage, fileStorage, child) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala new file mode 100644 index 000000000000..c658bc8c6da3 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.File +import java.net.URI +import java.util.Properties + +import scala.language.existentials + +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.mapred._ + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.internal.HiveSerDe +import org.apache.spark.util.{SerializableJobConf, Utils} + + +case class InsertIntoDirCommand(path: String, + isLocal: Boolean, + rowStorage: CatalogStorageFormat, + fileStorage: CatalogStorageFormat, + query: LogicalPlan) extends SaveAsHiveFile { + + override def children: Seq[LogicalPlan] = query :: Nil + + override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { + assert(children.length == 1) + + val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) => + r(0) = r(0) + a.name + "," + r(1) = r(1) + a.dataType.catalogString + ":" + r + } + + val properties = new Properties() + properties.put("columns", cols.dropRight(1)) + properties.put("columns.types", types.dropRight(1)) + + val sqlContext = sparkSession.sqlContext + + val defaultStorage: CatalogStorageFormat = { + val defaultStorageType = + sqlContext.conf.getConfString("hive.default.fileformat", "textfile") + val defaultHiveSerde = HiveSerDe.sourceToSerDe(defaultStorageType) + CatalogStorageFormat( + locationUri = None, + inputFormat = defaultHiveSerde.flatMap(_.inputFormat) + .orElse(Some("org.apache.hadoop.mapred.TextInputFormat")), + outputFormat = defaultHiveSerde.flatMap(_.outputFormat) + .orElse(Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")), + serde = defaultHiveSerde.flatMap(_.serde), + compressed = false, + properties = Map()) + } + + val storage = CatalogStorageFormat( + locationUri = Some(new URI(path)), + inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), + outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat), + serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde), + compressed = false, + properties = rowStorage.properties ++ fileStorage.properties) + + properties.put(serdeConstants.SERIALIZATION_LIB, + storage.serde.getOrElse(classOf[LazySimpleSerDe].getName)) + + import scala.collection.JavaConverters._ + properties.putAll(rowStorage.properties.asJava) + properties.putAll(fileStorage.properties.asJava) + + var tableDesc = new TableDesc( + Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: InputFormat[_, _]]], + Utils.classForName(storage.outputFormat.get), + properties + ) + + val targetPath = new Path(path) + val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc( + targetPath.toString, tableDesc, false) + + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val jobConf = new JobConf(hadoopConf) + jobConf.set("mapreduce.fileoutputcommitter.marksuccessfuljobs", "false") + val jobConfSer = new SerializableJobConf(jobConf) + + FileSystem.get(jobConf).delete(targetPath, true) + + saveAsHiveFile( + sparkSession = sparkSession, + plan = children.head, + fileSinkConf = fileSinkConf, + outputLocation = path) + +// val outputPath = FileOutputFormat.getOutputPath(jobConf) +// if( isLocal ) { +// Utils.deleteRecursively(new File(path)) +// outputPath.getFileSystem(hadoopConf).copyToLocalFile(true, outputPath, targetPath) +// log.info(s"Copied results from ${outputPath} to local dir ${path}") +// } else { +// log.info(s"Results available at path ${outputPath}") +// } + + Seq.empty[Row] + } +} + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 46610f84dd82..ffe6715f5a10 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -32,14 +32,12 @@ import org.apache.hadoop.hive.ql.exec.TaskRunner import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.SparkException -import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.{CommandUtils, DataWritingCommand} -import org.apache.spark.sql.execution.datasources.FileFormatWriter +import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive.client.{HiveClientImpl, HiveVersion} @@ -80,7 +78,7 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean) extends DataWritingCommand { + ifPartitionNotExists: Boolean) extends SaveAsHiveFile { override def children: Seq[LogicalPlan] = query :: Nil @@ -234,10 +232,9 @@ case class InsertIntoHiveTable( override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { assert(children.length == 1) - val sessionState = sparkSession.sessionState val externalCatalog = sparkSession.sharedState.externalCatalog val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version - val hadoopConf = sessionState.newHadoopConf() + val hadoopConf = sparkSession.sessionState.newHadoopConf() val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive") @@ -257,20 +254,6 @@ case class InsertIntoHiveTable( val tmpLocation = getExternalTmpPath(tableLocation, hiveVersion, hadoopConf, stagingDir, scratchDir) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean - - if (isCompressed) { - // Please note that isCompressed, "mapreduce.output.fileoutputformat.compress", - // "mapreduce.output.fileoutputformat.compress.codec", and - // "mapreduce.output.fileoutputformat.compress.type" - // have no impact on ORC because it uses table properties to store compression information. - hadoopConf.set("mapreduce.output.fileoutputformat.compress", "true") - fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(hadoopConf - .get("mapreduce.output.fileoutputformat.compress.codec")) - fileSinkConf.setCompressType(hadoopConf - .get("mapreduce.output.fileoutputformat.compress.type")) - } val numDynamicPartitions = partition.values.count(_.isEmpty) val numStaticPartitions = partition.values.count(_.nonEmpty) @@ -332,11 +315,6 @@ case class InsertIntoHiveTable( case _ => // do nothing since table has no bucketing } - val committer = FileCommitProtocol.instantiate( - sparkSession.sessionState.conf.fileCommitProtocolClass, - jobId = java.util.UUID.randomUUID().toString, - outputPath = tmpLocation.toString) - val partitionAttributes = partitionColumnNames.takeRight(numDynamicPartitions).map { name => query.resolve(name :: Nil, sparkSession.sessionState.analyzer.resolver).getOrElse { throw new AnalysisException( @@ -344,17 +322,12 @@ case class InsertIntoHiveTable( }.asInstanceOf[Attribute] } - FileFormatWriter.write( + saveAsHiveFile( sparkSession = sparkSession, plan = children.head, - fileFormat = new HiveFileFormat(fileSinkConf), - committer = committer, - outputSpec = FileFormatWriter.OutputSpec(tmpLocation.toString, Map.empty), - hadoopConf = hadoopConf, - partitionColumns = partitionAttributes, - bucketSpec = None, - statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), - options = Map.empty) + fileSinkConf = fileSinkConf, + outputLocation = tmpLocation.toString, + partitionAttributes = partitionAttributes) if (partition.nonEmpty) { if (numDynamicPartitions > 0) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala new file mode 100644 index 000000000000..e7a21e9faaf5 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.command.DataWritingCommand +import org.apache.spark.sql.execution.datasources.FileFormatWriter +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} + +// Base trait from which all hive insert statement physical execution extends. +private[hive] trait SaveAsHiveFile extends DataWritingCommand { + + protected def saveAsHiveFile(sparkSession: SparkSession, + plan: SparkPlan, + fileSinkConf: FileSinkDesc, + outputLocation: String, + partitionAttributes: Seq[Attribute] = Nil, + bucketSpec: Option[BucketSpec] = None, + options: Map[String, String] = Map.empty): Unit = { + + val sessionState = sparkSession.sessionState + val hadoopConf = sessionState.newHadoopConf() + + val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean + if (isCompressed) { + // Please note that isCompressed, "mapreduce.output.fileoutputformat.compress", + // "mapreduce.output.fileoutputformat.compress.codec", and + // "mapreduce.output.fileoutputformat.compress.type" + // have no impact on ORC because it uses table properties to store compression information. + hadoopConf.set("mapreduce.output.fileoutputformat.compress", "true") + fileSinkConf.setCompressed(true) + fileSinkConf.setCompressCodec(hadoopConf + .get("mapreduce.output.fileoutputformat.compress.codec")) + fileSinkConf.setCompressType(hadoopConf + .get("mapreduce.output.fileoutputformat.compress.type")) + } + + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.fileCommitProtocolClass, + jobId = java.util.UUID.randomUUID().toString, + outputPath = outputLocation) + + FileFormatWriter.write( + sparkSession = sparkSession, + plan = plan, + fileFormat = new HiveFileFormat(fileSinkConf), + committer = committer, + outputSpec = FileFormatWriter.OutputSpec(outputLocation, Map.empty), + hadoopConf = hadoopConf, + partitionColumns = partitionAttributes, + bucketSpec = bucketSpec, + statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), + options = options) + } +} + From a975536ea3fed0674410ebc65538e134992779b1 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 16 Aug 2017 20:59:25 -0700 Subject: [PATCH 02/47] Add Unittests --- .../sql/hive/execution/SQLQuerySuite.scala | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) 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 ef3d9b27aad7..48475a42fb84 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 @@ -2039,4 +2039,80 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(setOfPath.size() == pathSizeToDeleteOnExit) } } + + test("insert overwrite to dir from hive metastore table") { + import org.apache.spark.util.Utils + + val path = Utils.createTempDir() + path.delete() + checkAnswer( + sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' SELECT * FROM src where key < 10"), + Seq.empty[Row]) + + checkAnswer( + sql(s"""INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' + |STORED AS orc + |SELECT * FROM src where key < 10""".stripMargin), + Seq.empty[Row]) + + // use orc data source to check the data of path is right. + sql( + s"""CREATE TEMPORARY TABLE orc_source + |USING org.apache.spark.sql.hive.orc + |OPTIONS ( + | PATH '${path.getCanonicalPath}' + |) + """.stripMargin) + checkAnswer( + sql("select * from orc_source"), + sql("select * from src where key < 10").collect() + ) + + Utils.deleteRecursively(path) + dropTempTable("orc_source") + } + + test("insert overwrite to dir from temp table") { + import org.apache.spark.util.Utils + + sparkContext + .parallelize(1 to 10) + .map(i => TestData(i, i.toString)) + .toDF() + .registerTempTable("test_insert_table") + + val path = Utils.createTempDir() + path.delete() + checkAnswer( + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin), + Seq.empty[Row]) + + checkAnswer( + sql(s""" + INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' + |STORED AS orc + |SELECT * FROM test_insert_table""".stripMargin), + Seq.empty[Row]) + + // use orc data source to check the data of path is right. + sql( + s"""CREATE TEMPORARY TABLE orc_source + |USING org.apache.spark.sql.hive.orc + |OPTIONS ( + | PATH '${path.getCanonicalPath}' + |) + """.stripMargin) + checkAnswer( + sql("select * from orc_source"), + sql("select * from test_insert_table").collect() + ) + Utils.deleteRecursively(path) + dropTempTable("test_insert_table") + dropTempTable("orc_source") + } } From a15bf4e0c12a37127788528f1e69b0462a0e287b Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 16 Aug 2017 22:23:34 -0700 Subject: [PATCH 03/47] fix local path --- .../hive/execution/InsertIntoDirCommand.scala | 46 +++++++++++-------- .../hive/execution/InsertIntoHiveTable.scala | 1 + .../sql/hive/execution/SaveAsHiveFile.scala | 4 +- 3 files changed, 31 insertions(+), 20 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala index c658bc8c6da3..b1bf92c82fd4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.hive.execution -import java.io.File import java.net.URI import java.util.Properties import scala.language.existentials import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.internal.HiveSerDe -import org.apache.spark.util.{SerializableJobConf, Utils} +import org.apache.spark.util.Utils case class InsertIntoDirCommand(path: String, @@ -75,8 +75,9 @@ case class InsertIntoDirCommand(path: String, properties = Map()) } + val pathUri = if (isLocal) Utils.resolveURI(path) else new URI(path) val storage = CatalogStorageFormat( - locationUri = Some(new URI(path)), + locationUri = Some(pathUri), inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat), serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde), @@ -96,32 +97,39 @@ case class InsertIntoDirCommand(path: String, properties ) - val targetPath = new Path(path) - val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc( - targetPath.toString, tableDesc, false) - val hadoopConf = sparkSession.sessionState.newHadoopConf() val jobConf = new JobConf(hadoopConf) - jobConf.set("mapreduce.fileoutputcommitter.marksuccessfuljobs", "false") - val jobConfSer = new SerializableJobConf(jobConf) - FileSystem.get(jobConf).delete(targetPath, true) + val targetPath = new Path(path) + val writeToPath = + if (isLocal) { + val localFileSystem = FileSystem.getLocal(jobConf) + val localPath = localFileSystem.makeQualified(targetPath) + if (localFileSystem.exists(localPath)) { + localFileSystem.delete(localPath, true) + } + localPath + } else { + val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf) + val dfs = qualifiedPath.getFileSystem(jobConf) + if (dfs.exists(qualifiedPath)) { + dfs.delete(qualifiedPath, true) + } else { + dfs.mkdirs(qualifiedPath.getParent) + } + qualifiedPath + } + + val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc( + writeToPath.toString, tableDesc, false) saveAsHiveFile( sparkSession = sparkSession, plan = children.head, + hadoopConf = hadoopConf, fileSinkConf = fileSinkConf, outputLocation = path) -// val outputPath = FileOutputFormat.getOutputPath(jobConf) -// if( isLocal ) { -// Utils.deleteRecursively(new File(path)) -// outputPath.getFileSystem(hadoopConf).copyToLocalFile(true, outputPath, targetPath) -// log.info(s"Copied results from ${outputPath} to local dir ${path}") -// } else { -// log.info(s"Results available at path ${outputPath}") -// } - Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index ffe6715f5a10..112946b510af 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -325,6 +325,7 @@ case class InsertIntoHiveTable( saveAsHiveFile( sparkSession = sparkSession, plan = children.head, + hadoopConf = hadoopConf, fileSinkConf = fileSinkConf, outputLocation = tmpLocation.toString, partitionAttributes = partitionAttributes) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index e7a21e9faaf5..26ce87c73b6e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import org.apache.hadoop.conf.Configuration + import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -31,6 +33,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { protected def saveAsHiveFile(sparkSession: SparkSession, plan: SparkPlan, + hadoopConf: Configuration, fileSinkConf: FileSinkDesc, outputLocation: String, partitionAttributes: Seq[Attribute] = Nil, @@ -38,7 +41,6 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { options: Map[String, String] = Map.empty): Unit = { val sessionState = sparkSession.sessionState - val hadoopConf = sessionState.newHadoopConf() val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean if (isCompressed) { From b9db02ecf130a8524fc9e864e1758117094f04cf Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 17 Aug 2017 11:50:50 -0700 Subject: [PATCH 04/47] fix style --- .../scala/org/apache/spark/sql/execution/SparkSqlParser.scala | 4 +++- .../main/scala/org/apache/spark/sql/hive/HiveStrategies.scala | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 7e007920cf51..aad799408cea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -20,11 +20,13 @@ package org.apache.spark.sql.execution import java.util.Locale import scala.collection.JavaConverters._ + import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.TerminalNode + import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 19b2a7f16837..694d5187d8ed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -21,6 +21,7 @@ import java.io.IOException import java.util.Locale import org.apache.hadoop.fs.{FileSystem, Path} + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ From e05624ff15092773c329182e85cd2f942151def0 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 17 Aug 2017 23:53:27 -0700 Subject: [PATCH 05/47] condense storage --- .../plans/logical/basicLogicalOperators.scala | 3 +- .../spark/sql/execution/SparkSqlParser.scala | 13 +++++++- .../spark/sql/hive/HiveStrategies.scala | 4 +-- .../hive/execution/InsertIntoDirCommand.scala | 32 ++----------------- 4 files changed, 17 insertions(+), 35 deletions(-) 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 f8543fc83cc1..7c8488060326 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 @@ -362,8 +362,7 @@ case class InsertIntoTable( case class InsertIntoDir( path: String, isLocal: Boolean, - rowStorage: CatalogStorageFormat, - fileStorage: CatalogStorageFormat, + storage: CatalogStorageFormat, child: LogicalPlan) extends LogicalPlan { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index aad799408cea..4ffc80cebc83 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1522,12 +1522,23 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) .getOrElse(CatalogStorageFormat.empty) + val defaultStorage = HiveSerDe.getDefaultStorage(conf) + + val path = string(ctx.path) + val storage = CatalogStorageFormat( + locationUri = Some(CatalogUtils.stringToURI(path)), + inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), + outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat), + serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde), + compressed = false, + properties = rowStorage.properties ++ fileStorage.properties) + tableIdent match { case Some(ti: TableIdentifier) => InsertIntoTable(UnresolvedRelation(ti), partitionKeys, query, ctx.OVERWRITE != null, ctx.EXISTS != null) case _ => - InsertIntoDir(string(ctx.path), ctx.LOCAL != null, rowStorage, fileStorage, query) + InsertIntoDir(path, ctx.LOCAL != null, storage, query) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 694d5187d8ed..22deca49d84f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -156,8 +156,8 @@ object HiveAnalysis extends Rule[LogicalPlan] { case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => CreateHiveTableAsSelectCommand(tableDesc, query, mode) - case InsertIntoDir(path, isLocal, rowStorage, fileStorage, child) => - InsertIntoDirCommand(path, isLocal, rowStorage, fileStorage, child) + case InsertIntoDir(path, isLocal, storage, child) => + InsertIntoDirCommand(path, isLocal, storage, child) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala index b1bf92c82fd4..80d2f0ee4ad8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.hive.execution -import java.net.URI import java.util.Properties import scala.language.existentials @@ -33,14 +32,12 @@ import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.util.Utils case class InsertIntoDirCommand(path: String, isLocal: Boolean, - rowStorage: CatalogStorageFormat, - fileStorage: CatalogStorageFormat, + storage: CatalogStorageFormat, query: LogicalPlan) extends SaveAsHiveFile { override def children: Seq[LogicalPlan] = query :: Nil @@ -60,36 +57,11 @@ case class InsertIntoDirCommand(path: String, val sqlContext = sparkSession.sqlContext - val defaultStorage: CatalogStorageFormat = { - val defaultStorageType = - sqlContext.conf.getConfString("hive.default.fileformat", "textfile") - val defaultHiveSerde = HiveSerDe.sourceToSerDe(defaultStorageType) - CatalogStorageFormat( - locationUri = None, - inputFormat = defaultHiveSerde.flatMap(_.inputFormat) - .orElse(Some("org.apache.hadoop.mapred.TextInputFormat")), - outputFormat = defaultHiveSerde.flatMap(_.outputFormat) - .orElse(Some("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat")), - serde = defaultHiveSerde.flatMap(_.serde), - compressed = false, - properties = Map()) - } - - val pathUri = if (isLocal) Utils.resolveURI(path) else new URI(path) - val storage = CatalogStorageFormat( - locationUri = Some(pathUri), - inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), - outputFormat = fileStorage.outputFormat.orElse(defaultStorage.outputFormat), - serde = rowStorage.serde.orElse(fileStorage.serde).orElse(defaultStorage.serde), - compressed = false, - properties = rowStorage.properties ++ fileStorage.properties) - properties.put(serdeConstants.SERIALIZATION_LIB, storage.serde.getOrElse(classOf[LazySimpleSerDe].getName)) import scala.collection.JavaConverters._ - properties.putAll(rowStorage.properties.asJava) - properties.putAll(fileStorage.properties.asJava) + properties.putAll(storage.properties.asJava) var tableDesc = new TableDesc( Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: InputFormat[_, _]]], From 7f5664dcf5e55f3cc2ad91a000bb2c8ab58f596a Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 18 Aug 2017 00:13:00 -0700 Subject: [PATCH 06/47] change InsertInto to InsertIntoTable --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 6 +++--- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 8 ++++---- .../org/apache/spark/sql/execution/SparkSqlParser.scala | 6 +++--- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 3c1753ab81b4..a2b63f7073ae 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -237,7 +237,7 @@ query : ctes? queryNoWith ; -insertInto +insertIntoTable : INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)? | INSERT INTO TABLE? tableIdentifier partitionSpec? | INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? @@ -326,7 +326,7 @@ resource ; queryNoWith - : insertInto? queryTerm queryOrganization #singleInsertQuery + : insertIntoTable? queryTerm queryOrganization #singleInsertQuery | fromClause multiInsertQueryBody+ #multiInsertQuery ; @@ -340,7 +340,7 @@ queryOrganization ; multiInsertQueryBody - : insertInto? + : insertIntoTable? querySpecification queryOrganization ; 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 efe7019ffcc0..16f8ba208d36 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 @@ -151,7 +151,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Add organization statements. optionalMap(body.queryOrganization)(withQueryResultClauses). // Add insert. - optionalMap(body.insertInto())(withInsertInto) + optionalMap(body.insertIntoTable())(withInsertIntoTable) } // If there are multiple INSERTS just UNION them together into one query. @@ -170,14 +170,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Add organization statements. optionalMap(ctx.queryOrganization)(withQueryResultClauses). // Add insert. - optionalMap(ctx.insertInto())(withInsertInto) + optionalMap(ctx.insertIntoTable())(withInsertIntoTable) } /** * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan. */ - protected def withInsertInto( - ctx: InsertIntoContext, + protected def withInsertIntoTable( + ctx: InsertIntoTableContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { val tableIdent = visitTableIdentifier(ctx.tableIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 4ffc80cebc83..b8759d189c88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1366,7 +1366,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } else { // CREATE VIEW ... AS INSERT INTO is not allowed. ctx.query.queryNoWith match { - case s: SingleInsertQueryContext if s.insertInto != null => + case s: SingleInsertQueryContext if s.insertIntoTable != null => operationNotAllowed("CREATE VIEW ... AS INSERT INTO", ctx) case _: MultiInsertQueryContext => operationNotAllowed("CREATE VIEW ... AS FROM ... [INSERT INTO ...]+", ctx) @@ -1505,8 +1505,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE or INSERT INTO [LOCAL] DIRECOTRY * operation to the logical plan. */ - protected override def withInsertInto(ctx: InsertIntoContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + protected override def withInsertIntoTable(ctx: InsertIntoTableContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { val tableIdent = Option(ctx.tableIdentifier).map(visitTableIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) From d50b3a2a722783e4bf971382946fce3e61ddee3b Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Sat, 19 Aug 2017 09:25:28 -0700 Subject: [PATCH 07/47] add InsertIntoDirectory --- .../spark/sql/catalyst/parser/SqlBase.g4 | 7 +- .../sql/catalyst/parser/AstBuilder.scala | 16 +++++ .../spark/sql/execution/SparkSqlParser.scala | 70 ++++++++++++++----- .../hive/execution/InsertIntoDirCommand.scala | 3 +- 4 files changed, 75 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index a2b63f7073ae..21d1c663a035 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -162,6 +162,7 @@ statement | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration + | insertOverwriteDirectory #insertOverwriteDir | unsupportedHiveNativeCommands .*? #failNativeCommand ; @@ -240,7 +241,11 @@ query insertIntoTable : INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)? | INSERT INTO TABLE? tableIdentifier partitionSpec? - | INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? + ; + +insertOverwriteDirectory + : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? query #insertOverwriteHiveDir + | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? (OPTIONS options=tablePropertyList) query #insertOverwriteDir ; partitionSpecLocation 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 16f8ba208d36..297086f8f3f6 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 @@ -196,6 +196,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging ctx.EXISTS != null) } + /** + * Write to a file, returning a [[InsertIntoDir]] logical plan. + */ + override def visitInsertOverwriteDir( + ctx: InsertOverwriteDirContext): LogicalPlan = withOrigin(ctx) { + throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx) + } + + /** + * Write to a file, returning a [[InsertIntoDir]] logical plan. + */ + override def visitInsertOverwriteHiveDir( + ctx: InsertOverwriteHiveDirContext): LogicalPlan = withOrigin(ctx) { + throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx) + } + /** * Create a partition specification map. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index b8759d189c88..7fdcfac05be6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1502,26 +1502,64 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } /** - * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE or INSERT INTO [LOCAL] DIRECOTRY - * operation to the logical plan. - */ - protected override def withInsertIntoTable(ctx: InsertIntoTableContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - val tableIdent = Option(ctx.tableIdentifier).map(visitTableIdentifier) - val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) - - val dynamicPartitionKeys = 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) + * Write to a file, returning a [[InsertIntoDir]] logical plan. + * + * Expected format: + * {{{ + * INSERT OVERWRITE DIRECTORY + * [path] + * [OPTIONS table_property_list] + * select_statement; + * }}} + */ + override def visitInsertOverwriteDir( + ctx: InsertOverwriteDirContext): LogicalPlan = withOrigin(ctx) { + val options = visitPropertyKeyValues(ctx.options) + var storage = DataSource.buildStorageFormatFromOptions(options) + + val path = string(ctx.path) + if (!path.isEmpty && storage.locationUri.isDefined) { + throw new ParseException( + "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " + + "you can only specify one of them.", ctx) } + if (path.isEmpty && !storage.locationUri.isDefined) { + throw new ParseException( + "You need to specify directory path or 'path' in OPTIONS, but not both", ctx) + } + val customLocation = storage.locationUri.orElse(Some(CatalogUtils.stringToURI(path))) + + storage = storage.copy(locationUri = customLocation) + InsertIntoDir(path, ctx.LOCAL != null, storage, plan(ctx.query)) + } + + /** + * Write to a file, returning a [[InsertIntoDir]] logical plan. + * + * Expected format: + * {{{ + * INSERT OVERWRITE DIRECTORY + * path + * [ROW FORMAT row_format] + * [STORED AS file_format] + * select_statement; + * }}} + */ + override def visitInsertOverwriteHiveDir( + ctx: InsertOverwriteHiveDirContext): LogicalPlan = withOrigin(ctx) { validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) val rowStorage = Option(ctx.rowFormat).map(visitRowFormat) .getOrElse(CatalogStorageFormat.empty) val fileStorage = Option(ctx.createFileFormat).map(visitCreateFileFormat) .getOrElse(CatalogStorageFormat.empty) + val path = string(ctx.path) + // The path field is required + if (path.isEmpty) { + operationNotAllowed("INSERT OVERWRITE DIRECTORY must be accompanied by path", ctx) + } + val defaultStorage = HiveSerDe.getDefaultStorage(conf) val path = string(ctx.path) @@ -1533,12 +1571,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { compressed = false, properties = rowStorage.properties ++ fileStorage.properties) - tableIdent match { - case Some(ti: TableIdentifier) => - InsertIntoTable(UnresolvedRelation(ti), partitionKeys, query, - ctx.OVERWRITE != null, ctx.EXISTS != null) - case _ => - InsertIntoDir(path, ctx.LOCAL != null, storage, query) - } + InsertIntoDir(path, ctx.LOCAL != null, storage, plan(ctx.query)) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala index 80d2f0ee4ad8..48e3dacd21cf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala @@ -44,6 +44,7 @@ case class InsertIntoDirCommand(path: String, override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { assert(children.length == 1) + assert(!storage.locationUri.isEmpty) val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) => r(0) = r(0) + a.name + "," @@ -72,7 +73,7 @@ case class InsertIntoDirCommand(path: String, val hadoopConf = sparkSession.sessionState.newHadoopConf() val jobConf = new JobConf(hadoopConf) - val targetPath = new Path(path) + val targetPath = new Path(storage.locationUri.get) val writeToPath = if (isLocal) { val localFileSystem = FileSystem.getLocal(jobConf) From 61a18a29955e71a89eecfeab6a01ba725be157ce Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Sat, 19 Aug 2017 09:36:32 -0700 Subject: [PATCH 08/47] update insertInto --- .../apache/spark/sql/catalyst/parser/SqlBase.g4 | 14 +++++++++----- .../spark/sql/catalyst/parser/AstBuilder.scala | 6 +++--- .../spark/sql/execution/SparkSqlParser.scala | 2 +- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 21d1c663a035..7f900dc21e3b 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -162,7 +162,6 @@ statement | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration - | insertOverwriteDirectory #insertOverwriteDir | unsupportedHiveNativeCommands .*? #failNativeCommand ; @@ -244,8 +243,13 @@ insertIntoTable ; insertOverwriteDirectory - : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? query #insertOverwriteHiveDir - | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? (OPTIONS options=tablePropertyList) query #insertOverwriteDir + : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir + | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? (OPTIONS options=tablePropertyList) #insertOverwriteDir + ; + +insertInto + : insertIntoTable + | insertOverwriteDirectory ; partitionSpecLocation @@ -331,7 +335,7 @@ resource ; queryNoWith - : insertIntoTable? queryTerm queryOrganization #singleInsertQuery + : insertInto? queryTerm queryOrganization #singleInsertQuery | fromClause multiInsertQueryBody+ #multiInsertQuery ; @@ -345,7 +349,7 @@ queryOrganization ; multiInsertQueryBody - : insertIntoTable? + : insertInto? querySpecification queryOrganization ; 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 297086f8f3f6..9b81bd847aae 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 @@ -151,7 +151,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Add organization statements. optionalMap(body.queryOrganization)(withQueryResultClauses). // Add insert. - optionalMap(body.insertIntoTable())(withInsertIntoTable) + optionalMap(body.insertIntoTable())(plan(ctx.insertInto)) } // If there are multiple INSERTS just UNION them together into one query. @@ -170,13 +170,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Add organization statements. optionalMap(ctx.queryOrganization)(withQueryResultClauses). // Add insert. - optionalMap(ctx.insertIntoTable())(withInsertIntoTable) + optionalMap(ctx.insertIntoTable())(plan(ctx.insertInto)) } /** * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan. */ - protected def withInsertIntoTable( + private def withInsertIntoTable( ctx: InsertIntoTableContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { val tableIdent = visitTableIdentifier(ctx.tableIdentifier) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 7fdcfac05be6..1dfcfe8c975c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1366,7 +1366,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } else { // CREATE VIEW ... AS INSERT INTO is not allowed. ctx.query.queryNoWith match { - case s: SingleInsertQueryContext if s.insertIntoTable != null => + case s: SingleInsertQueryContext if s.insertInto != null => operationNotAllowed("CREATE VIEW ... AS INSERT INTO", ctx) case _: MultiInsertQueryContext => operationNotAllowed("CREATE VIEW ... AS FROM ... [INSERT INTO ...]+", ctx) From 4c19aafff3167206eaf9895f66ae5bf5301725cf Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Sat, 19 Aug 2017 11:16:57 -0700 Subject: [PATCH 09/47] SQLQuerySuite passed --- .../sql/catalyst/parser/AstBuilder.scala | 38 +++++++++++++++++-- .../plans/logical/basicLogicalOperators.scala | 1 - .../spark/sql/execution/SparkSqlParser.scala | 31 ++++++++------- .../spark/sql/hive/HiveStrategies.scala | 7 ++-- .../hive/execution/InsertIntoDirCommand.scala | 10 ++--- 5 files changed, 58 insertions(+), 29 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 9b81bd847aae..4cc6a628aa8a 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 @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ @@ -151,7 +152,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Add organization statements. optionalMap(body.queryOrganization)(withQueryResultClauses). // Add insert. - optionalMap(body.insertIntoTable())(plan(ctx.insertInto)) + optionalMap(body.insertInto())(withInsertInto) } // If there are multiple INSERTS just UNION them together into one query. @@ -170,7 +171,36 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Add organization statements. optionalMap(ctx.queryOrganization)(withQueryResultClauses). // Add insert. - optionalMap(ctx.insertIntoTable())(plan(ctx.insertInto)) + optionalMap(ctx.insertInto())(withInsertInto) + } + + /** + * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat). + */ + type InsertDirParams = (Boolean, CatalogStorageFormat) + + private def withInsertInto( + ctx: InsertIntoContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + assert(ctx.children.size == 1) + + ctx.getChild(0) match { + case c if c. isInstanceOf[InsertIntoTableContext] => + withInsertIntoTable(c.asInstanceOf[InsertIntoTableContext], query) + case c if c.isInstanceOf[InsertOverwriteDirectoryContext] => + withInsertOverwriteDirectory(c.asInstanceOf[InsertOverwriteDirectoryContext], query) + } + } + + private def withInsertOverwriteDirectory( + ctx: InsertOverwriteDirectoryContext, + query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + val (isLocal, storage) = ctx match { + case dir: InsertOverwriteDirContext => visitInsertOverwriteDir(dir) + case hiveDir: InsertOverwriteHiveDirContext => visitInsertOverwriteHiveDir(hiveDir) + } + + InsertIntoDir(isLocal, storage, query) } /** @@ -200,7 +230,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Write to a file, returning a [[InsertIntoDir]] logical plan. */ override def visitInsertOverwriteDir( - ctx: InsertOverwriteDirContext): LogicalPlan = withOrigin(ctx) { + ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) { throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx) } @@ -208,7 +238,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Write to a file, returning a [[InsertIntoDir]] logical plan. */ override def visitInsertOverwriteHiveDir( - ctx: InsertOverwriteHiveDirContext): LogicalPlan = withOrigin(ctx) { + ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) { throw new ParseException("INSERT OVERWRITE DIRECTORY is not supported", ctx) } 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 7c8488060326..7e6a5f868588 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 @@ -360,7 +360,6 @@ case class InsertIntoTable( } case class InsertIntoDir( - path: String, isLocal: Boolean, storage: CatalogStorageFormat, child: LogicalPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 1dfcfe8c975c..ddab0eff5609 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1513,7 +1513,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * }}} */ override def visitInsertOverwriteDir( - ctx: InsertOverwriteDirContext): LogicalPlan = withOrigin(ctx) { + ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) { val options = visitPropertyKeyValues(ctx.options) var storage = DataSource.buildStorageFormatFromOptions(options) @@ -1531,23 +1531,23 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { storage = storage.copy(locationUri = customLocation) - InsertIntoDir(path, ctx.LOCAL != null, storage, plan(ctx.query)) + (ctx.LOCAL != null, storage) } /** - * Write to a file, returning a [[InsertIntoDir]] logical plan. - * - * Expected format: - * {{{ - * INSERT OVERWRITE DIRECTORY - * path - * [ROW FORMAT row_format] - * [STORED AS file_format] - * select_statement; - * }}} - */ + * Write to a file, returning a [[InsertIntoDir]] logical plan. + * + * Expected format: + * {{{ + * INSERT OVERWRITE DIRECTORY + * path + * [ROW FORMAT row_format] + * [STORED AS file_format] + * select_statement; + * }}} + */ override def visitInsertOverwriteHiveDir( - ctx: InsertOverwriteHiveDirContext): LogicalPlan = withOrigin(ctx) { + ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) { validateRowFormatFileFormat(ctx.rowFormat, ctx.createFileFormat, ctx) val rowStorage = Option(ctx.rowFormat).map(visitRowFormat) .getOrElse(CatalogStorageFormat.empty) @@ -1562,7 +1562,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { val defaultStorage = HiveSerDe.getDefaultStorage(conf) - val path = string(ctx.path) val storage = CatalogStorageFormat( locationUri = Some(CatalogUtils.stringToURI(path)), inputFormat = fileStorage.inputFormat.orElse(defaultStorage.inputFormat), @@ -1571,6 +1570,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { compressed = false, properties = rowStorage.properties ++ fileStorage.properties) - InsertIntoDir(path, ctx.LOCAL != null, storage, plan(ctx.query)) + (ctx.LOCAL != null, storage) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 22deca49d84f..a39c5beb572d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -26,7 +26,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, ScriptTransformation} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, + ScriptTransformation} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} @@ -156,8 +157,8 @@ object HiveAnalysis extends Rule[LogicalPlan] { case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => CreateHiveTableAsSelectCommand(tableDesc, query, mode) - case InsertIntoDir(path, isLocal, storage, child) => - InsertIntoDirCommand(path, isLocal, storage, child) + case InsertIntoDir(isLocal, storage, child) => + InsertIntoDirCommand(isLocal, storage, child) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala index 48e3dacd21cf..266595b22446 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala @@ -35,10 +35,10 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.Utils -case class InsertIntoDirCommand(path: String, - isLocal: Boolean, - storage: CatalogStorageFormat, - query: LogicalPlan) extends SaveAsHiveFile { +case class InsertIntoDirCommand( + isLocal: Boolean, + storage: CatalogStorageFormat, + query: LogicalPlan) extends SaveAsHiveFile { override def children: Seq[LogicalPlan] = query :: Nil @@ -101,7 +101,7 @@ case class InsertIntoDirCommand(path: String, plan = children.head, hadoopConf = hadoopConf, fileSinkConf = fileSinkConf, - outputLocation = path) + outputLocation = targetPath.toString) Seq.empty[Row] } From 47fde8a6591cbdad5656b51c3b55e3647d7b8d1f Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Sat, 19 Aug 2017 11:37:39 -0700 Subject: [PATCH 10/47] fix comments --- .../apache/spark/sql/catalyst/parser/AstBuilder.scala | 10 ++++++++++ .../apache/spark/sql/execution/SparkSqlParser.scala | 7 +++---- 2 files changed, 13 insertions(+), 4 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 4cc6a628aa8a..410392f61d18 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 @@ -179,6 +179,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ type InsertDirParams = (Boolean, CatalogStorageFormat) + /** + * Add an + * INSERT INTO [TABLE] or + * INSERT OVERWRITE TABLE or + * INSERT OVERWRITE [LOCAL] DIRECTORY + * operation to logical plan + */ private def withInsertInto( ctx: InsertIntoContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { @@ -192,6 +199,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Add an INSERT OVERWRITE [LOCAL] DIRECTORY operation to the logical plan + */ private def withInsertOverwriteDirectory( ctx: InsertOverwriteDirectoryContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index ddab0eff5609..a91c884fc5ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -26,7 +26,6 @@ import org.antlr.v4.runtime.tree.TerminalNode import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser._ @@ -1502,7 +1501,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } /** - * Write to a file, returning a [[InsertIntoDir]] logical plan. + * Return the parameters for [[InsertIntoDir]] logical plan. * * Expected format: * {{{ @@ -1527,15 +1526,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { throw new ParseException( "You need to specify directory path or 'path' in OPTIONS, but not both", ctx) } - val customLocation = storage.locationUri.orElse(Some(CatalogUtils.stringToURI(path))) + val customLocation = storage.locationUri.orElse(Some(CatalogUtils.stringToURI(path))) storage = storage.copy(locationUri = customLocation) (ctx.LOCAL != null, storage) } /** - * Write to a file, returning a [[InsertIntoDir]] logical plan. + * Return the parameters for [[InsertIntoDir]] logical plan. * * Expected format: * {{{ From da7065ba5b30a777878ee1f8cb6c5fb4cfa42212 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 21 Aug 2017 09:35:14 -0700 Subject: [PATCH 11/47] Add tableProdier --- .../org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 4 ++-- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 8 ++++---- .../catalyst/plans/logical/basicLogicalOperators.scala | 1 + .../org/apache/spark/sql/execution/SparkSqlParser.scala | 6 ++++-- .../sql/execution/datasources/DataSourceStrategy.scala | 2 ++ .../scala/org/apache/spark/sql/hive/HiveStrategies.scala | 4 ++-- ...ntoDirCommand.scala => InsertIntoHiveDirCommand.scala} | 2 +- 7 files changed, 16 insertions(+), 11 deletions(-) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/{InsertIntoDirCommand.scala => InsertIntoHiveDirCommand.scala} (99%) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 7f900dc21e3b..d91fe1ade583 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -243,8 +243,8 @@ insertIntoTable ; insertOverwriteDirectory - : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir - | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? (OPTIONS options=tablePropertyList) #insertOverwriteDir + : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir + | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList) #insertOverwriteDir ; insertInto 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 410392f61d18..cccc50e4bf52 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 @@ -175,9 +175,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat). + * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). */ - type InsertDirParams = (Boolean, CatalogStorageFormat) + type InsertDirParams = (Boolean, CatalogStorageFormat, Option[String]) /** * Add an @@ -205,12 +205,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging private def withInsertOverwriteDirectory( ctx: InsertOverwriteDirectoryContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - val (isLocal, storage) = ctx match { + val (isLocal, storage, provider) = ctx match { case dir: InsertOverwriteDirContext => visitInsertOverwriteDir(dir) case hiveDir: InsertOverwriteHiveDirContext => visitInsertOverwriteHiveDir(hiveDir) } - InsertIntoDir(isLocal, storage, query) + InsertIntoDir(isLocal, storage, provider, query) } /** 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 7e6a5f868588..cbec94257e3b 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 @@ -362,6 +362,7 @@ case class InsertIntoTable( case class InsertIntoDir( isLocal: Boolean, storage: CatalogStorageFormat, + provider: Option[String], child: LogicalPlan) extends LogicalPlan { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 863da9a31928..731d0a817233 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1540,7 +1540,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { val customLocation = storage.locationUri.orElse(Some(CatalogUtils.stringToURI(path))) storage = storage.copy(locationUri = customLocation) - (ctx.LOCAL != null, storage) + val provider = ctx.tableProvider.qualifiedName.getText + + (ctx.LOCAL != null, storage, Some(provider)) } /** @@ -1579,6 +1581,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { compressed = false, properties = rowStorage.properties ++ fileStorage.properties) - (ctx.LOCAL != null, storage) + (ctx.LOCAL != null, storage, None) } } 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 237017742770..f955be4e8af4 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 @@ -202,6 +202,8 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast mode, table, Some(t.location)) + // case InsertIntoDir(isLocal, storage, child) => + // InsertIntoDirCommand(isLocal, storage, child) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index a39c5beb572d..b6900c2d1f9c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -157,8 +157,8 @@ object HiveAnalysis extends Rule[LogicalPlan] { case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => CreateHiveTableAsSelectCommand(tableDesc, query, mode) - case InsertIntoDir(isLocal, storage, child) => - InsertIntoDirCommand(isLocal, storage, child) + case InsertIntoDir(isLocal, storage, _, child) => + InsertIntoHiveDirCommand(isLocal, storage, child) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala similarity index 99% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 266595b22446..44f0e0b4f765 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.Utils -case class InsertIntoDirCommand( +case class InsertIntoHiveDirCommand( isLocal: Boolean, storage: CatalogStorageFormat, query: LogicalPlan) extends SaveAsHiveFile { From 7f4b488a3aff850949c4d414eea8c071265a7d4c Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 21 Aug 2017 13:54:46 -0700 Subject: [PATCH 12/47] Add InsertIntoDataSourceDirCommand --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 16 +++-- .../InsertIntoDataSourceDirCommand.scala | 65 +++++++++++++++++++ .../datasources/DataSourceStrategy.scala | 6 +- .../execution/command/DDLCommandSuite.scala | 48 +++++++++++++- .../sql/execution/command/DDLSuite.scala | 31 +++++++++ 6 files changed, 159 insertions(+), 11 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index d91fe1ade583..5222ba933d84 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -243,8 +243,8 @@ insertIntoTable ; insertOverwriteDirectory - : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir - | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList) #insertOverwriteDir + : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir + | INSERT OVERWRITE DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)? #insertOverwriteDir ; insertInto diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 731d0a817233..205690b71356 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1523,10 +1523,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { */ override def visitInsertOverwriteDir( ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) { - val options = visitPropertyKeyValues(ctx.options) + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) var storage = DataSource.buildStorageFormatFromOptions(options) - val path = string(ctx.path) + val path = Option(ctx.path) match { + case Some(s) => string(s) + case None => "" + } + if (!path.isEmpty && storage.locationUri.isDefined) { throw new ParseException( "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " + @@ -1537,12 +1541,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { "You need to specify directory path or 'path' in OPTIONS, but not both", ctx) } - val customLocation = storage.locationUri.orElse(Some(CatalogUtils.stringToURI(path))) - storage = storage.copy(locationUri = customLocation) + if (!path.isEmpty) { + val customLocation = Some(CatalogUtils.stringToURI(path)) + storage = storage.copy(locationUri = customLocation) + } val provider = ctx.tableProvider.qualifiedName.getText - (ctx.LOCAL != null, storage, Some(provider)) + (false, storage, Some(provider)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala new file mode 100644 index 000000000000..e6c226df6eba --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources._ + +/** + * A command used to write the result of a query to a directory. + * + * The syntax of using this command in SQL is: + * {{{ + * INSERT OVERWRITE DIRECTORY (path=STRING)? + * USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...]) + * SELECT ... + * }}} + */ +case class InsertIntoDataSourceDirCommand( + storage: CatalogStorageFormat, + provider: Option[String], + query: LogicalPlan) extends RunnableCommand { + + override def innerChildren: Seq[LogicalPlan] = Seq(query) + + override def run(sparkSession: SparkSession): Seq[Row] = { + assert(innerChildren.length == 1) + assert(!storage.locationUri.isEmpty) + assert(provider.isDefined) + + // Create the relation based on the input logical plan: `data`. + val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) + val dataSource = DataSource( + sparkSession, + className = provider.get, + options = storage.properties ++ pathOption, + catalogTable = None) + + try { + dataSource.writeAndRead(SaveMode.Overwrite, query) + } catch { + case ex: AnalysisException => + logError(s"Failed to write to directory " + storage.locationUri.toString, ex) + throw ex + } + + Seq.empty[Row] + } +} 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 f955be4e8af4..94423c4a8b3c 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 @@ -29,7 +29,8 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, + Project} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} @@ -140,6 +141,9 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast parts, query, overwrite, false) if parts.isEmpty => InsertIntoDataSourceCommand(l, query, overwrite) + case InsertIntoDir(_, storage, provider, query) => + InsertIntoDataSourceDirCommand(storage, provider, query) + case i @ InsertIntoTable( l @ LogicalRelation(t: HadoopFsRelation, _, table), parts, query, overwrite, _) => // If the InsertIntoTable command is for a partitioned HadoopFsRelation and diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 5643c58d9f84..140b63ac603d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -20,13 +20,12 @@ package org.apache.spark.sql.execution.command import java.net.URI import java.util.Locale -import scala.reflect.{classTag, ClassTag} - +import scala.reflect.{ClassTag, classTag} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, Project} import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -487,6 +486,49 @@ class DDLCommandSuite extends PlanTest { assert(e.message.contains("you can only specify one of them.")) } + test("insert overwrite directory") { + val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a" + parser.parsePlan(v1) match { + case InsertIntoDir(_, storage, provider, query) => + assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file") + case other => + fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + + " from query," + s"got ${other.getClass.getName}: $v1") + } + + val v2 = "INSERT OVERWRITE DIRECTORY USING parquet SELECT 1 as a" + val e2 = intercept[ParseException] { + parser.parsePlan(v2) + } + assert(e2.message.contains("You need to specify directory path or 'path' in OPTIONS")) + + val v3 = + """ + | INSERT OVERWRITE DIRECTORY USING json + | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE) + | SELECT 1 as a + """.stripMargin + parser.parsePlan(v3) match { + case InsertIntoDir(_, storage, provider, query) => + assert(storage.locationUri != None && provider == Some("json")) + case other => + fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + + " from query," + s"got ${other.getClass.getName}: $v1") + } + + val v4 = + """ + | INSERT OVERWRITE DIRECTORY '/tmp/file' USING json + | OPTIONS ('path' '/tmp/file', a 1, b 0.1, c TRUE) + | SELECT 1 as a + """.stripMargin + val e4 = intercept[ParseException] { + parser.parsePlan(v4) + } + assert(e4.message.contains( + "Directory path and 'path' in OPTIONS are both used to indicate the directory path")) + } + // ALTER TABLE table_name RENAME TO new_table_name; // ALTER VIEW view_name RENAME TO new_view_name; test("alter table/view: rename table/view") { 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 9332f773430e..067f3ff607a6 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 @@ -2346,6 +2346,37 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } + test("insert overwrite directory") { + case class ClassData(a: Int, b: String) + + val path = Utils.createTempDir() + path.delete() + + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY '${path.toString}' USING json + | OPTIONS (a 1, b 0.1, c TRUE) + | SELECT 1 as a, 'c' as b + """.stripMargin + checkAnswer( + spark.sql(v1), + Seq.empty[Row]) + + // use orc data source to check the data of path is right. + sql( + s"""CREATE TEMPORARY TABLE json_source + |USING json + |OPTIONS ( + | PATH '${path.getCanonicalPath}' + |) + """.stripMargin) + checkAnswer( + sql("select * from json_source"), + sql("SELECT 1 as a, 'c' as b") + ) + Utils.deleteRecursively(path) + } + Seq(true, false).foreach { caseSensitive => test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { From 73f605ed955657e6db4d29503ea1df403c304ba4 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 21 Aug 2017 13:59:12 -0700 Subject: [PATCH 13/47] fix style --- .../apache/spark/sql/execution/command/DDLCommandSuite.scala | 3 ++- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 2 -- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 140b63ac603d..83cb79781598 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.execution.command import java.net.URI import java.util.Locale -import scala.reflect.{ClassTag, classTag} +import scala.reflect.{classTag, ClassTag} + import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser.ParseException 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 067f3ff607a6..0b1be42f63bc 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 @@ -2347,8 +2347,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } test("insert overwrite directory") { - case class ClassData(a: Int, b: String) - val path = Utils.createTempDir() path.delete() From 8261b39aa4a0ad31a62ebcdb33c718a121f9fbbc Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 21 Aug 2017 14:04:24 -0700 Subject: [PATCH 14/47] fix style --- .../scala/org/apache/spark/sql/execution/SparkSqlParser.scala | 4 ++-- .../spark/sql/execution/datasources/DataSourceStrategy.scala | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 205690b71356..27913d009f1f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -30,9 +30,9 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, _} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{CreateTable, _} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} import org.apache.spark.sql.types.StructType 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 94423c4a8b3c..eb4fe6859385 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 @@ -206,8 +206,6 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast mode, table, Some(t.location)) - // case InsertIntoDir(isLocal, storage, child) => - // InsertIntoDirCommand(isLocal, storage, child) } } From 0882dd1f3c300f832d731b69a0d57ef461e55038 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 21 Aug 2017 22:48:09 -0700 Subject: [PATCH 15/47] Address gatorsmile's comments --- .../UnsupportedOperationChecker.scala | 1 - .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 3 +- .../spark/sql/execution/SparkSqlParser.scala | 2 +- .../InsertIntoDataSourceDirCommand.scala | 8 +- .../datasources/DataSourceStrategy.scala | 4 +- .../execution/command/DDLCommandSuite.scala | 4 +- .../spark/sql/hive/HiveStrategies.scala | 4 +- .../execution/InsertIntoHiveDirCommand.scala | 17 ++- .../sql/hive/execution/SaveAsHiveFile.scala | 17 +-- .../sql/hive/execution/SQLQuerySuite.scala | 123 +++++++++--------- 11 files changed, 102 insertions(+), 83 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index f8a8f6c7fe4d..33ba0867a33e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -142,7 +142,6 @@ object UnsupportedOperationChecker { "Distinct aggregations are not supported on streaming DataFrames/Datasets. Consider " + "using approx_count_distinct() instead.") - case _: Command => throwError("Commands like CreateTable*, AlterTable*, Show* are not supported with " + "streaming DataFrames/Datasets") 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 cccc50e4bf52..a0cad6a4b79f 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 @@ -210,7 +210,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case hiveDir: InsertOverwriteHiveDirContext => visitInsertOverwriteHiveDir(hiveDir) } - InsertIntoDir(isLocal, storage, provider, query) + InsertIntoDir(isLocal, storage, provider, query, overwrite = true) } /** 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 cbec94257e3b..e7e04a6500dd 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 @@ -363,7 +363,8 @@ case class InsertIntoDir( isLocal: Boolean, storage: CatalogStorageFormat, provider: Option[String], - child: LogicalPlan) + child: LogicalPlan, + overwrite: Boolean = true) extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 27913d009f1f..25321a9a1b68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1556,7 +1556,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * * Expected format: * {{{ - * INSERT OVERWRITE DIRECTORY + * INSERT OVERWRITE [LOCAL] DIRECTORY * path * [ROW FORMAT row_format] * [STORED AS file_format] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index e6c226df6eba..ef2f2c74d7b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -35,13 +35,14 @@ import org.apache.spark.sql.execution.datasources._ case class InsertIntoDataSourceDirCommand( storage: CatalogStorageFormat, provider: Option[String], - query: LogicalPlan) extends RunnableCommand { + query: LogicalPlan, + overwrite: Boolean) extends RunnableCommand { override def innerChildren: Seq[LogicalPlan] = Seq(query) override def run(sparkSession: SparkSession): Seq[Row] = { assert(innerChildren.length == 1) - assert(!storage.locationUri.isEmpty) + assert(storage.locationUri.nonEmpty) assert(provider.isDefined) // Create the relation based on the input logical plan: `data`. @@ -52,8 +53,9 @@ case class InsertIntoDataSourceDirCommand( options = storage.properties ++ pathOption, catalogTable = None) + val saveMode = if (overwrite) SaveMode.Overwrite else SaveMode.ErrorIfExists try { - dataSource.writeAndRead(SaveMode.Overwrite, query) + dataSource.writeAndRead(saveMode, query) } catch { case ex: AnalysisException => logError(s"Failed to write to directory " + storage.locationUri.toString, ex) 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 eb4fe6859385..cb846b64ad2f 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 @@ -141,8 +141,8 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast parts, query, overwrite, false) if parts.isEmpty => InsertIntoDataSourceCommand(l, query, overwrite) - case InsertIntoDir(_, storage, provider, query) => - InsertIntoDataSourceDirCommand(storage, provider, query) + case InsertIntoDir(_, storage, provider, query, overwrite) if provider.nonEmpty => + InsertIntoDataSourceDirCommand(storage, provider, query, overwrite) case i @ InsertIntoTable( l @ LogicalRelation(t: HadoopFsRelation, _, table), parts, query, overwrite, _) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 83cb79781598..dd5fb2071e17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -490,7 +490,7 @@ class DDLCommandSuite extends PlanTest { test("insert overwrite directory") { val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a" parser.parsePlan(v1) match { - case InsertIntoDir(_, storage, provider, query) => + case InsertIntoDir(_, storage, provider, query, overwrite) => assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file") case other => fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + @@ -510,7 +510,7 @@ class DDLCommandSuite extends PlanTest { | SELECT 1 as a """.stripMargin parser.parsePlan(v3) match { - case InsertIntoDir(_, storage, provider, query) => + case InsertIntoDir(_, storage, provider, query, overwrite) => assert(storage.locationUri != None && provider == Some("json")) case other => fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index b6900c2d1f9c..03923f068cf1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -157,8 +157,8 @@ object HiveAnalysis extends Rule[LogicalPlan] { case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => CreateHiveTableAsSelectCommand(tableDesc, query, mode) - case InsertIntoDir(isLocal, storage, _, child) => - InsertIntoHiveDirCommand(isLocal, storage, child) + case InsertIntoDir(isLocal, storage, _, child, overwrite) => + InsertIntoHiveDirCommand(isLocal, storage, child, overwrite) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 44f0e0b4f765..5f376bd49f5a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -38,13 +38,14 @@ import org.apache.spark.util.Utils case class InsertIntoHiveDirCommand( isLocal: Boolean, storage: CatalogStorageFormat, - query: LogicalPlan) extends SaveAsHiveFile { + query: LogicalPlan, + overwrite: Boolean) extends SaveAsHiveFile { override def children: Seq[LogicalPlan] = query :: Nil override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { assert(children.length == 1) - assert(!storage.locationUri.isEmpty) + assert(storage.locationUri.nonEmpty) val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) => r(0) = r(0) + a.name + "," @@ -79,14 +80,22 @@ case class InsertIntoHiveDirCommand( val localFileSystem = FileSystem.getLocal(jobConf) val localPath = localFileSystem.makeQualified(targetPath) if (localFileSystem.exists(localPath)) { - localFileSystem.delete(localPath, true) + if (overwrite) { + localFileSystem.delete(localPath, true) + } else { + throw new RuntimeException("Directory '" + localPath.toString + "' already exists") + } } localPath } else { val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf) val dfs = qualifiedPath.getFileSystem(jobConf) if (dfs.exists(qualifiedPath)) { - dfs.delete(qualifiedPath, true) + if (overwrite) { + dfs.delete(qualifiedPath, true) + } else { + throw new RuntimeException("Directory '" + qualifiedPath.toString + "' already exists") + } } else { dfs.mkdirs(qualifiedPath.getParent) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index 26ce87c73b6e..f3927952529e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -31,14 +31,15 @@ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} // Base trait from which all hive insert statement physical execution extends. private[hive] trait SaveAsHiveFile extends DataWritingCommand { - protected def saveAsHiveFile(sparkSession: SparkSession, - plan: SparkPlan, - hadoopConf: Configuration, - fileSinkConf: FileSinkDesc, - outputLocation: String, - partitionAttributes: Seq[Attribute] = Nil, - bucketSpec: Option[BucketSpec] = None, - options: Map[String, String] = Map.empty): Unit = { + protected def saveAsHiveFile( + sparkSession: SparkSession, + plan: SparkPlan, + hadoopConf: Configuration, + fileSinkConf: FileSinkDesc, + outputLocation: String, + partitionAttributes: Seq[Attribute] = Nil, + bucketSpec: Option[BucketSpec] = None, + options: Map[String, String] = Map.empty): Unit = { val sessionState = sparkSession.sessionState 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 2593c7ffcf70..f4a4bfaabfe4 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 @@ -2046,74 +2046,81 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val path = Utils.createTempDir() path.delete() - checkAnswer( - sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' SELECT * FROM src where key < 10"), - Seq.empty[Row]) + withTempDir { dir => + val path = dir.toURI.getPath - checkAnswer( - sql(s"""INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' + checkAnswer( + sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"), + Seq.empty[Row]) + + checkAnswer( + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' |STORED AS orc - |SELECT * FROM src where key < 10""".stripMargin), - Seq.empty[Row]) + |SELECT * FROM src where key < 10 + """.stripMargin), + Seq.empty[Row]) - // use orc data source to check the data of path is right. - sql( - s"""CREATE TEMPORARY TABLE orc_source - |USING org.apache.spark.sql.hive.orc - |OPTIONS ( - | PATH '${path.getCanonicalPath}' - |) - """.stripMargin) - checkAnswer( - sql("select * from orc_source"), - sql("select * from src where key < 10").collect() - ) + // use orc data source to check the data of path is right. + sql( + s""" + |CREATE TEMPORARY TABLE orc_source + |USING org.apache.spark.sql.hive.orc + |OPTIONS ( + | PATH '${dir.getCanonicalPath}' + |) + """.stripMargin) - Utils.deleteRecursively(path) - dropTempTable("orc_source") + checkAnswer( + sql("select * from orc_source"), + sql("select * from src where key < 10").collect()) + + dropTempTable("orc_source") + } } test("insert overwrite to dir from temp table") { - import org.apache.spark.util.Utils + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") - sparkContext - .parallelize(1 to 10) - .map(i => TestData(i, i.toString)) - .toDF() - .registerTempTable("test_insert_table") + withTempDir { dir => + val path = dir.toURI.getPath - val path = Utils.createTempDir() - path.delete() - checkAnswer( - sql( - s""" - |INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' - |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - |SELECT * FROM test_insert_table - """.stripMargin), - Seq.empty[Row]) + checkAnswer( + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin), + Seq.empty[Row]) - checkAnswer( - sql(s""" - INSERT OVERWRITE LOCAL DIRECTORY '${path.toString}' - |STORED AS orc - |SELECT * FROM test_insert_table""".stripMargin), - Seq.empty[Row]) + checkAnswer( + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |STORED AS orc + |SELECT * FROM test_insert_table + """.stripMargin), + Seq.empty[Row]) - // use orc data source to check the data of path is right. - sql( - s"""CREATE TEMPORARY TABLE orc_source - |USING org.apache.spark.sql.hive.orc - |OPTIONS ( - | PATH '${path.getCanonicalPath}' - |) - """.stripMargin) - checkAnswer( - sql("select * from orc_source"), - sql("select * from test_insert_table").collect() - ) - Utils.deleteRecursively(path) - dropTempTable("test_insert_table") - dropTempTable("orc_source") + // use orc data source to check the data of path is right. + sql( + s""" + |CREATE TEMPORARY TABLE orc_source + |USING org.apache.spark.sql.hive.orc + |OPTIONS ( + | PATH '${dir.getCanonicalPath}' + |) + """.stripMargin) + + checkAnswer( + sql("select * from orc_source"), + sql("select * from test_insert_table").collect()) + + dropTempTable("orc_source") + } + } } } From c813ad8a072d96a1cbedcce7df7d168affe84c5f Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 21 Aug 2017 22:56:09 -0700 Subject: [PATCH 16/47] Use withTempDir --- .../sql/execution/command/DDLSuite.scala | 53 ++++++++++--------- .../sql/hive/execution/SQLQuerySuite.scala | 4 -- 2 files changed, 28 insertions(+), 29 deletions(-) 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 92388777dc8f..b896147db269 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 @@ -2347,32 +2347,35 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } test("insert overwrite directory") { - val path = Utils.createTempDir() - path.delete() - - val v1 = - s""" - | INSERT OVERWRITE DIRECTORY '${path.toString}' USING json - | OPTIONS (a 1, b 0.1, c TRUE) - | SELECT 1 as a, 'c' as b - """.stripMargin - checkAnswer( - spark.sql(v1), - Seq.empty[Row]) + withTempDir { dir => + val path = dir.toURI.getPath - // use orc data source to check the data of path is right. - sql( - s"""CREATE TEMPORARY TABLE json_source - |USING json - |OPTIONS ( - | PATH '${path.getCanonicalPath}' - |) - """.stripMargin) - checkAnswer( - sql("select * from json_source"), - sql("SELECT 1 as a, 'c' as b") - ) - Utils.deleteRecursively(path) + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY '${path}' + | USING json + | OPTIONS (a 1, b 0.1, c TRUE) + | SELECT 1 as a, 'c' as b + """.stripMargin + + checkAnswer( + spark.sql(v1), + Seq.empty[Row]) + + // use orc data source to check the data of path is right. + sql( + s""" + |CREATE TEMPORARY TABLE json_source + |USING json + |OPTIONS ( + | PATH '${dir.getCanonicalPath}' + |) + """.stripMargin) + + checkAnswer( + sql("select * from json_source"), + sql("SELECT 1 as a, 'c' as b")) + } } Seq(true, false).foreach { caseSensitive => 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 f4a4bfaabfe4..767065e57147 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 @@ -2042,10 +2042,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("insert overwrite to dir from hive metastore table") { - import org.apache.spark.util.Utils - - val path = Utils.createTempDir() - path.delete() withTempDir { dir => val path = dir.toURI.getPath From bc5424cba17a72f5717b5ce6a2816bd7ab069b82 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Mon, 21 Aug 2017 22:56:46 -0700 Subject: [PATCH 17/47] line length --- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 b896147db269..248a935c79c1 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 @@ -26,7 +26,8 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, + NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.internal.SQLConf From f36e933f47163b0f9040087e4aeee582e0ccd9a4 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 23 Aug 2017 13:29:09 -0700 Subject: [PATCH 18/47] Address gatorsmile's comment --- .../plans/logical/basicLogicalOperators.scala | 9 +++ .../spark/sql/execution/SparkSqlParser.scala | 7 +- .../InsertIntoDataSourceDirCommand.scala | 5 +- .../sql/execution/command/DDLSuite.scala | 24 +++--- .../sql/hive/execution/SaveAsHiveFile.scala | 4 +- ...HiveTableSuite.scala => InsertSuite.scala} | 81 ++++++++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 79 ------------------ 7 files changed, 110 insertions(+), 99 deletions(-) rename sql/hive/src/test/scala/org/apache/spark/sql/hive/{InsertIntoHiveTableSuite.scala => InsertSuite.scala} (88%) 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 e1212b919d60..9a5e7616aad4 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 @@ -359,6 +359,15 @@ case class InsertIntoTable( override lazy val resolved: Boolean = false } +/** + * Insert query result into a directory. + * + * @param isLocal Indicates whether the specified directory is local directory + * @param storage Info about output file, row and what serialization format + * @param provider Specifies what data source to use; only used for data source file. + * @param child The query to be executed + * @param overwrite If true, the existing directory will be overwritten + */ case class InsertIntoDir( isLocal: Boolean, storage: CatalogStorageFormat, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 25321a9a1b68..dd00e1e892f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1526,17 +1526,14 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) var storage = DataSource.buildStorageFormatFromOptions(options) - val path = Option(ctx.path) match { - case Some(s) => string(s) - case None => "" - } + val path = Option(ctx.path).map(string).getOrElse("") if (!path.isEmpty && storage.locationUri.isDefined) { throw new ParseException( "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " + "you can only specify one of them.", ctx) } - if (path.isEmpty && !storage.locationUri.isDefined) { + if (path.isEmpty && storage.locationUri.isEmpty) { throw new ParseException( "You need to specify directory path or 'path' in OPTIONS, but not both", ctx) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index ef2f2c74d7b0..99477dc981a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -42,8 +42,8 @@ case class InsertIntoDataSourceDirCommand( override def run(sparkSession: SparkSession): Seq[Row] = { assert(innerChildren.length == 1) - assert(storage.locationUri.nonEmpty) - assert(provider.isDefined) + assert(storage.locationUri.nonEmpty, "Directory path is required") + assert(provider.isDefined, "Data source is required") // Create the relation based on the input logical plan: `data`. val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) @@ -55,6 +55,7 @@ case class InsertIntoDataSourceDirCommand( val saveMode = if (overwrite) SaveMode.Overwrite else SaveMode.ErrorIfExists try { + sparkSession.sessionState.executePlan(dataSource.planForWriting(saveMode, query)) dataSource.writeAndRead(saveMode, query) } catch { case ex: AnalysisException => 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 248a935c79c1..942be9d04a68 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 @@ -2364,18 +2364,20 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { Seq.empty[Row]) // use orc data source to check the data of path is right. - sql( - s""" - |CREATE TEMPORARY TABLE json_source - |USING json - |OPTIONS ( - | PATH '${dir.getCanonicalPath}' - |) - """.stripMargin) + withTempView("orc_source") { + sql( + s""" + |CREATE TEMPORARY TABLE json_source + |USING json + |OPTIONS ( + | PATH '${dir.getCanonicalPath}' + |) + """.stripMargin) - checkAnswer( - sql("select * from json_source"), - sql("SELECT 1 as a, 'c' as b")) + checkAnswer( + sql("select * from json_source"), + sql("SELECT 1 as a, 'c' as b")) + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index f3927952529e..e1eab08e505c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -62,10 +62,12 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { jobId = java.util.UUID.randomUUID().toString, outputPath = outputLocation) + val fileFormat = new HiveFileFormat(fileSinkConf) + FileFormatWriter.write( sparkSession = sparkSession, plan = plan, - fileFormat = new HiveFileFormat(fileSinkConf), + fileFormat = fileFormat, committer = committer, outputSpec = FileFormatWriter.OutputSpec(outputLocation, Map.empty), hadoopConf = hadoopConf, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala similarity index 88% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index cc80f2e481cb..56bb6d663e4a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -34,7 +34,7 @@ case class TestData(key: Int, value: String) case class ThreeCloumntable(key: Int, value: String, key1: String) -class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter +class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter with SQLTestUtils { import spark.implicits._ @@ -534,4 +534,83 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef } } } + + test("insert overwrite to dir from hive metastore table") { + withTempDir { dir => + val path = dir.toURI.getPath + + checkAnswer( + sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"), + Seq.empty[Row]) + + checkAnswer( + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |STORED AS orc + |SELECT * FROM src where key < 10 + """.stripMargin), + Seq.empty[Row]) + + // use orc data source to check the data of path is right. + withTempView("orc_source") { + sql( + s""" + |CREATE TEMPORARY TABLE orc_source + |USING org.apache.spark.sql.hive.orc + |OPTIONS ( + | PATH '${dir.getCanonicalPath}' + |) + """.stripMargin) + + checkAnswer( + sql("select * from orc_source"), + sql("select * from src where key < 10").collect()) + } + } + } + + test("insert overwrite to dir from temp table") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + withTempDir { dir => + val path = dir.toURI.getPath + + checkAnswer( + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin), + Seq.empty[Row]) + + checkAnswer( + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |STORED AS orc + |SELECT * FROM test_insert_table + """.stripMargin), + Seq.empty[Row]) + + // use orc data source to check the data of path is right. +// withTempView("orc_source") { +// sql( +// s""" +// |CREATE TEMPORARY VIEW orc_source +// |USING org.apache.spark.sql.hive.orc +// |OPTIONS ( +// | PATH '${dir.getCanonicalPath}' +// |) +// """.stripMargin) +// +// checkAnswer( +// sql("select * from orc_source"), +// sql("select * from test_insert_table").collect()) +// } + } + } + } } 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 134dec6b8088..d2a6ef7b2b37 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 @@ -2000,83 +2000,4 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { assert(setOfPath.size() == pathSizeToDeleteOnExit) } } - - test("insert overwrite to dir from hive metastore table") { - withTempDir { dir => - val path = dir.toURI.getPath - - checkAnswer( - sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"), - Seq.empty[Row]) - - checkAnswer( - sql( - s""" - |INSERT OVERWRITE LOCAL DIRECTORY '${path}' - |STORED AS orc - |SELECT * FROM src where key < 10 - """.stripMargin), - Seq.empty[Row]) - - // use orc data source to check the data of path is right. - sql( - s""" - |CREATE TEMPORARY TABLE orc_source - |USING org.apache.spark.sql.hive.orc - |OPTIONS ( - | PATH '${dir.getCanonicalPath}' - |) - """.stripMargin) - - checkAnswer( - sql("select * from orc_source"), - sql("select * from src where key < 10").collect()) - - dropTempTable("orc_source") - } - } - - test("insert overwrite to dir from temp table") { - withTempView("test_insert_table") { - spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") - - withTempDir { dir => - val path = dir.toURI.getPath - - checkAnswer( - sql( - s""" - |INSERT OVERWRITE LOCAL DIRECTORY '${path}' - |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - |SELECT * FROM test_insert_table - """.stripMargin), - Seq.empty[Row]) - - checkAnswer( - sql( - s""" - |INSERT OVERWRITE LOCAL DIRECTORY '${path}' - |STORED AS orc - |SELECT * FROM test_insert_table - """.stripMargin), - Seq.empty[Row]) - - // use orc data source to check the data of path is right. - sql( - s""" - |CREATE TEMPORARY TABLE orc_source - |USING org.apache.spark.sql.hive.orc - |OPTIONS ( - | PATH '${dir.getCanonicalPath}' - |) - """.stripMargin) - - checkAnswer( - sql("select * from orc_source"), - sql("select * from test_insert_table").collect()) - - dropTempTable("orc_source") - } - } - } } From 64f37f4a2517fb841a47d6fb105e01f10e700fc3 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 23 Aug 2017 13:33:32 -0700 Subject: [PATCH 19/47] fix typo --- .../sql/hive/execution/SaveAsHiveFile.scala | 4 +-- .../apache/spark/sql/hive/InsertSuite.scala | 30 +++++++++---------- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index e1eab08e505c..f3927952529e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -62,12 +62,10 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { jobId = java.util.UUID.randomUUID().toString, outputPath = outputLocation) - val fileFormat = new HiveFileFormat(fileSinkConf) - FileFormatWriter.write( sparkSession = sparkSession, plan = plan, - fileFormat = fileFormat, + fileFormat = new HiveFileFormat(fileSinkConf), committer = committer, outputSpec = FileFormatWriter.OutputSpec(outputLocation, Map.empty), hadoopConf = hadoopConf, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 56bb6d663e4a..f3ad3a87b2b4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -556,7 +556,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter withTempView("orc_source") { sql( s""" - |CREATE TEMPORARY TABLE orc_source + |CREATE TEMPORARY VIEW orc_source |USING org.apache.spark.sql.hive.orc |OPTIONS ( | PATH '${dir.getCanonicalPath}' @@ -596,20 +596,20 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter Seq.empty[Row]) // use orc data source to check the data of path is right. -// withTempView("orc_source") { -// sql( -// s""" -// |CREATE TEMPORARY VIEW orc_source -// |USING org.apache.spark.sql.hive.orc -// |OPTIONS ( -// | PATH '${dir.getCanonicalPath}' -// |) -// """.stripMargin) -// -// checkAnswer( -// sql("select * from orc_source"), -// sql("select * from test_insert_table").collect()) -// } + withTempView("orc_source") { + sql( + s""" + |CREATE TEMPORARY VIEW orc_source + |USING org.apache.spark.sql.hive.orc + |OPTIONS ( + | PATH '${dir.getCanonicalPath}' + |) + """.stripMargin) + + checkAnswer( + sql("select * from orc_source"), + sql("select * from test_insert_table").collect()) + } } } } From 8ebe5e22ef0883038a72a9c12d5a9afd8f373652 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 31 Aug 2017 13:23:31 -0700 Subject: [PATCH 20/47] Address gatorsmile's comments --- .../spark/sql/execution/SparkSqlParser.scala | 2 +- .../InsertIntoDataSourceDirCommand.scala | 6 +- .../datasources/DataSourceStrategy.scala | 9 +- .../sql/execution/command/DDLSuite.scala | 4 +- .../spark/sql/hive/HiveStrategies.scala | 3 +- .../execution/InsertIntoHiveDirCommand.scala | 9 +- .../apache/spark/sql/hive/InsertSuite.scala | 84 +++++++++++++------ 7 files changed, 78 insertions(+), 39 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index dd00e1e892f6..f5951c00e203 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1584,6 +1584,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { compressed = false, properties = rowStorage.properties ++ fileStorage.properties) - (ctx.LOCAL != null, storage, None) + (ctx.LOCAL != null, storage, Some(DDLUtils.HIVE_PROVIDER)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index 99477dc981a4..c4c47168346a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.datasources._ */ case class InsertIntoDataSourceDirCommand( storage: CatalogStorageFormat, - provider: Option[String], + provider: String, query: LogicalPlan, overwrite: Boolean) extends RunnableCommand { @@ -43,13 +43,13 @@ case class InsertIntoDataSourceDirCommand( override def run(sparkSession: SparkSession): Seq[Row] = { assert(innerChildren.length == 1) assert(storage.locationUri.nonEmpty, "Directory path is required") - assert(provider.isDefined, "Data source is required") + assert(!provider.isEmpty, "Data source is required") // Create the relation based on the input logical plan: `data`. val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) val dataSource = DataSource( sparkSession, - className = provider.get, + className = provider, options = storage.properties ++ pathOption, catalogTable = None) 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 c17116c37a2a..8f455cec28ac 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources +import java.util.Locale import java.util.concurrent.Callable import org.apache.spark.internal.Logging @@ -29,8 +30,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, - Project} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, Project} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} @@ -141,8 +141,9 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast parts, query, overwrite, false) if parts.isEmpty => InsertIntoDataSourceCommand(l, query, overwrite) - case InsertIntoDir(_, storage, provider, query, overwrite) if provider.nonEmpty => - InsertIntoDataSourceDirCommand(storage, provider, query, overwrite) + case InsertIntoDir(_, storage, provider, query, overwrite) + if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) != DDLUtils.HIVE_PROVIDER => + InsertIntoDataSourceDirCommand(storage, provider.get, query, overwrite) case i @ InsertIntoTable( l @ LogicalRelation(t: HadoopFsRelation, _, table, _), parts, query, overwrite, _) => 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 942be9d04a68..200a08ac4cb7 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 @@ -2359,9 +2359,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { | SELECT 1 as a, 'c' as b """.stripMargin - checkAnswer( - spark.sql(v1), - Seq.empty[Row]) + spark.sql(v1) // use orc data source to check the data of path is right. withTempView("orc_source") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 03923f068cf1..43013696bf49 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -157,7 +157,8 @@ object HiveAnalysis extends Rule[LogicalPlan] { case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => CreateHiveTableAsSelectCommand(tableDesc, query, mode) - case InsertIntoDir(isLocal, storage, _, child, overwrite) => + case InsertIntoDir(isLocal, storage, provider, child, overwrite) + if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER => InsertIntoHiveDirCommand(isLocal, storage, child, overwrite) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 5f376bd49f5a..0a402267e5d0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -34,7 +34,14 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.Utils - +/** + * Command for writing the results of `query` to file system. + * + * @param isLocal whether the path specified in `storage` is a local directory + * @param storage storage format used to describe how the query result is stored. + * @param query the logical plan representing data to write to + * @param overwrite whthere overwrites existing directory + */ case class InsertIntoHiveDirCommand( isLocal: Boolean, storage: CatalogStorageFormat, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index f3ad3a87b2b4..4bf16cdf118a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -539,18 +539,14 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter withTempDir { dir => val path = dir.toURI.getPath - checkAnswer( - sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10"), - Seq.empty[Row]) + sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src where key < 10") - checkAnswer( - sql( - s""" - |INSERT OVERWRITE LOCAL DIRECTORY '${path}' - |STORED AS orc - |SELECT * FROM src where key < 10 - """.stripMargin), - Seq.empty[Row]) + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |STORED AS orc + |SELECT * FROM src where key < 10 + """.stripMargin) // use orc data source to check the data of path is right. withTempView("orc_source") { @@ -570,30 +566,66 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } - test("insert overwrite to dir from temp table") { + test("insert overwrite to local dir from temp table") { withTempView("test_insert_table") { spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") withTempDir { dir => val path = dir.toURI.getPath - checkAnswer( - sql( - s""" - |INSERT OVERWRITE LOCAL DIRECTORY '${path}' - |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - |SELECT * FROM test_insert_table - """.stripMargin), - Seq.empty[Row]) + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) - checkAnswer( + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY '${path}' + |STORED AS orc + |SELECT * FROM test_insert_table + """.stripMargin) + + // use orc data source to check the data of path is right. + withTempView("orc_source") { sql( s""" - |INSERT OVERWRITE LOCAL DIRECTORY '${path}' - |STORED AS orc - |SELECT * FROM test_insert_table - """.stripMargin), - Seq.empty[Row]) + |CREATE TEMPORARY VIEW orc_source + |USING org.apache.spark.sql.hive.orc + |OPTIONS ( + | PATH '${dir.getCanonicalPath}' + |) + """.stripMargin) + + checkAnswer( + sql("select * from orc_source"), + sql("select * from test_insert_table").collect()) + } + } + } + } + + test("insert overwrite to dir from temp table") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + withTempDir { dir => + val pathUri = dir.toURI + + sql( + s""" + |INSERT OVERWRITE DIRECTORY '${pathUri}' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + + sql( + s""" + |INSERT OVERWRITE DIRECTORY '${pathUri}' + |STORED AS orc + |SELECT * FROM test_insert_table + """.stripMargin) // use orc data source to check the data of path is right. withTempView("orc_source") { From 51f9a0a7a2dd539dd22e5c9baa8324e215b21750 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 31 Aug 2017 13:36:17 -0700 Subject: [PATCH 21/47] fix comments and style --- .../command/InsertIntoDataSourceDirCommand.scala | 2 +- .../apache/spark/sql/execution/command/DDLSuite.scala | 2 +- .../sql/hive/execution/InsertIntoHiveDirCommand.scala | 9 +++++++++ 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index c4c47168346a..271e21705c14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -45,7 +45,7 @@ case class InsertIntoDataSourceDirCommand( assert(storage.locationUri.nonEmpty, "Directory path is required") assert(!provider.isEmpty, "Data source is required") - // Create the relation based on the input logical plan: `data`. + // Create the relation based on the input logical plan: `query`. val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) val dataSource = DataSource( sparkSession, 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 200a08ac4cb7..4883c5c97a15 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 @@ -2359,7 +2359,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { | SELECT 1 as a, 'c' as b """.stripMargin - spark.sql(v1) + spark.sql(v1) // use orc data source to check the data of path is right. withTempView("orc_source") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 0a402267e5d0..dde9d74f2e73 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -37,6 +37,15 @@ import org.apache.spark.util.Utils /** * Command for writing the results of `query` to file system. * + * The syntax of using this command in SQL is: + * {{{ + * INSERT OVERWRITE [LOCAL] DIRECTORY + * path + * [ROW FORMAT row_format] + * [STORED AS file_format] + * SELECT ... + * }}} + * * @param isLocal whether the path specified in `storage` is a local directory * @param storage storage format used to describe how the query result is stored. * @param query the logical plan representing data to write to From e2db5e1e0cc491480828328e07b7bb619dc05bbd Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 31 Aug 2017 13:48:34 -0700 Subject: [PATCH 22/47] add more comments --- .../execution/command/InsertIntoDataSourceDirCommand.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index 271e21705c14..f3d4833c03e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -31,6 +31,11 @@ import org.apache.spark.sql.execution.datasources._ * USING format OPTIONS ([option1_name "option1_value", option2_name "option2_value", ...]) * SELECT ... * }}} + * + * @param storage storage format used to describe how the query result is stored. + * @param provider the data source type to be used + * @param query the logical plan representing data to write to + * @param overwrite whthere overwrites existing directory */ case class InsertIntoDataSourceDirCommand( storage: CatalogStorageFormat, From 7ccbde47fffba7bef8eceba4993bbd70eeb84845 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 1 Sep 2017 18:06:46 -0700 Subject: [PATCH 23/47] Address gatorsmile's comments --- .../spark/sql/catalyst/parser/SqlBase.g4 | 4 +- .../spark/sql/execution/SparkSqlParser.scala | 5 ++ .../InsertIntoDataSourceDirCommand.scala | 9 +++ .../spark/sql/execution/command/ddl.scala | 20 ++++++- .../datasources/DataSourceStrategy.scala | 16 ++--- .../sql/execution/command/DDLSuite.scala | 26 ++++++-- .../spark/sql/hive/HiveStrategies.scala | 4 ++ .../execution/InsertIntoHiveDirCommand.scala | 59 +++++++++++-------- .../apache/spark/sql/hive/InsertSuite.scala | 17 ++++++ 9 files changed, 120 insertions(+), 40 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 81036886b9fe..d7e075783ddb 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -247,8 +247,8 @@ insertIntoTable ; insertOverwriteDirectory - : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir - | INSERT OVERWRITE DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)? #insertOverwriteDir + : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir + | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)? #insertOverwriteDir ; insertInto diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index f5951c00e203..9e54c20438b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1523,6 +1523,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { */ override def visitInsertOverwriteDir( ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) { + if (ctx.LOCAL != null) { + throw new ParseException( + "LOCAL is not supported in INSERT OVERWRITE DIRECTORY to data source", ctx) + } + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) var storage = DataSource.buildStorageFormatFromOptions(options) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index f3d4833c03e8..1d4ff42c1198 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.command +import org.apache.hadoop.fs.FileSystem +import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -52,12 +54,19 @@ case class InsertIntoDataSourceDirCommand( // Create the relation based on the input logical plan: `query`. val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) + val dataSource = DataSource( sparkSession, className = provider, options = storage.properties ++ pathOption, catalogTable = None) + val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass) + if (!isFileFormat) { + throw new SparkException( + "Only Data Sources providing FileFormat are supported.") + } + val saveMode = if (overwrite) SaveMode.Overwrite else SaveMode.ErrorIfExists try { sparkSession.sessionState.executePlan(dataSource.planForWriting(saveMode, query)) 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 dae160f1bbb1..1eb861682363 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 @@ -22,18 +22,17 @@ import java.util.Locale import scala.collection.{GenMap, GenSeq} import scala.collection.parallel.ForkJoinTaskSupport import scala.util.control.NonFatal - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ 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.{NoSuchTableException, Resolver} 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.PartitioningUtils +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} import org.apache.spark.sql.types._ import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} @@ -848,4 +847,19 @@ object DDLUtils { } } } + + + /** + * Throws exception if outputPath tries to overwrite inputpath. + */ + def verifyNotReadPath(query: LogicalPlan, outputPath: Path) : Unit = { + val inputPaths = query.collect { + case LogicalRelation(r: HadoopFsRelation, _, _, _) => r.location.rootPaths + }.flatten + + if (inputPaths.contains(outputPath)) { + throw new AnalysisException( + "Cannot overwrite a path that is also being read from.") + } + } } 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 8f455cec28ac..c17b6feb72c8 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale import java.util.concurrent.Callable +import org.apache.hadoop.fs.Path + import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ @@ -141,8 +143,12 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast parts, query, overwrite, false) if parts.isEmpty => InsertIntoDataSourceCommand(l, query, overwrite) - case InsertIntoDir(_, storage, provider, query, overwrite) + case InsertIntoDir(isLocal, storage, provider, query, overwrite) if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) != DDLUtils.HIVE_PROVIDER => + + val outputPath = new Path(storage.locationUri.get) + if (overwrite) DDLUtils.verifyNotReadPath(query, outputPath) + InsertIntoDataSourceDirCommand(storage, provider.get, query, overwrite) case i @ InsertIntoTable( @@ -181,15 +187,9 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast } val outputPath = t.location.rootPaths.head - val inputPaths = actualQuery.collect { - case LogicalRelation(r: HadoopFsRelation, _, _, _) => r.location.rootPaths - }.flatten + if (overwrite) DDLUtils.verifyNotReadPath(actualQuery, outputPath) val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append - if (overwrite && inputPaths.contains(outputPath)) { - throw new AnalysisException( - "Cannot overwrite a path that is also being read from.") - } val partitionSchema = actualQuery.resolve( t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver) 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 4883c5c97a15..c287afdadb70 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 @@ -22,17 +22,16 @@ import java.net.URI import java.util.Locale import org.apache.hadoop.fs.Path +import org.apache.spark.SparkException import org.scalatest.BeforeAndAfterEach - import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, - NoSuchTableException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.{SQLTestUtils, SharedSQLContext} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -2379,6 +2378,25 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } + test("insert overwrite directory to data source not providing FileFormat") { + withTempDir { dir => + val path = dir.toURI.getPath + + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY '${path}' + | USING JDBC + | OPTIONS (a 1, b 0.1, c TRUE) + | SELECT 1 as a, 'c' as b + """.stripMargin + val e = intercept[SparkException] { + spark.sql(v1) + }.getMessage + + assert(e.contains("Only Data Sources providing FileFormat are supported")) + } + } + Seq(true, false).foreach { caseSensitive => test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 43013696bf49..f59a2c042a68 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -159,6 +159,10 @@ object HiveAnalysis extends Rule[LogicalPlan] { case InsertIntoDir(isLocal, storage, provider, child, overwrite) if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER => + + val outputPath = new Path(storage.locationUri.get) + if (overwrite) DDLUtils.verifyNotReadPath(child, outputPath) + InsertIntoHiveDirCommand(isLocal, storage, child, overwrite) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index dde9d74f2e73..4035d4bca862 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -20,18 +20,19 @@ package org.apache.spark.sql.hive.execution import java.util.Properties import scala.language.existentials - import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.mapred._ - +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.hive.client.HiveClientImpl +import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils /** @@ -63,28 +64,40 @@ case class InsertIntoHiveDirCommand( assert(children.length == 1) assert(storage.locationUri.nonEmpty) - val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) => - r(0) = r(0) + a.name + "," - r(1) = r(1) + a.dataType.catalogString + ":" - r - } - - val properties = new Properties() - properties.put("columns", cols.dropRight(1)) - properties.put("columns.types", types.dropRight(1)) - - val sqlContext = sparkSession.sqlContext - - properties.put(serdeConstants.SERIALIZATION_LIB, +// val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) => +// r(0) = r(0) + a.name + "," +// r(1) = r(1) + a.dataType.catalogString + ":" +// r +// } +// +// val properties = new Properties() +// properties.put("columns", cols.dropRight(1)) +// properties.put("columns.types", types.dropRight(1)) +// properties.put(serdeConstants.SERIALIZATION_LIB, +// storage.serde.getOrElse(classOf[LazySimpleSerDe].getName)) +// +// import scala.collection.JavaConverters._ +// properties.putAll(storage.properties.asJava) +// +// val tableDesc = new TableDesc( +// Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: InputFormat[_, _]]], +// Utils.classForName(storage.outputFormat.get), +// properties +// ) + + val hiveTable = HiveClientImpl.toHiveTable(CatalogTable( + identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")), + tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW, + storage = storage, + schema = query.schema + )) + hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB, storage.serde.getOrElse(classOf[LazySimpleSerDe].getName)) - import scala.collection.JavaConverters._ - properties.putAll(storage.properties.asJava) - - var tableDesc = new TableDesc( - Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: InputFormat[_, _]]], - Utils.classForName(storage.outputFormat.get), - properties + val tableDesc = new TableDesc( + hiveTable.getInputFormatClass, + hiveTable.getOutputFormatClass, + hiveTable.getMetadata ) val hadoopConf = sparkSession.sessionState.newHadoopConf() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 4bf16cdf118a..c88d6cf7cac5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -645,4 +645,21 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } } + + test("insert overwrite to dir to illegal path") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + val e = intercept[IllegalArgumentException] { + sql( + s""" + |INSERT OVERWRITE LOCAL DIRECTORY 'abc://a' + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + }.getMessage + + assert(e.contains("Wrong FS: abc://a, expected: file:///")) + } + } } From 52350e8adb7b07540f635c103f36e065c151f8cd Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 1 Sep 2017 18:52:40 -0700 Subject: [PATCH 24/47] fix style --- .../command/InsertIntoDataSourceDirCommand.scala | 1 - .../org/apache/spark/sql/execution/command/ddl.scala | 2 ++ .../apache/spark/sql/execution/command/DDLSuite.scala | 5 +++-- .../sql/hive/execution/InsertIntoHiveDirCommand.scala | 10 ++++------ 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index 1d4ff42c1198..3f8ed16204fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.command -import org.apache.hadoop.fs.FileSystem import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ 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 1eb861682363..25688320c147 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 @@ -22,9 +22,11 @@ import java.util.Locale import scala.collection.{GenMap, GenSeq} import scala.collection.parallel.ForkJoinTaskSupport import scala.util.control.NonFatal + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ 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.{NoSuchTableException, Resolver} 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 c287afdadb70..7164e18adb5e 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 @@ -22,8 +22,9 @@ import java.net.URI import java.util.Locale import org.apache.hadoop.fs.Path -import org.apache.spark.SparkException import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} @@ -31,7 +32,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION -import org.apache.spark.sql.test.{SQLTestUtils, SharedSQLContext} +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 4035d4bca862..682a3753ea24 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -17,23 +17,21 @@ package org.apache.spark.sql.hive.execution -import java.util.Properties - import scala.language.existentials + import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.mapred._ -import org.apache.spark.sql.catalyst.TableIdentifier + import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.hive.client.HiveClientImpl -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.Utils /** * Command for writing the results of `query` to file system. From 2ec9947af571691966e979e17aec12d3d683decf Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Sun, 3 Sep 2017 12:28:28 -0700 Subject: [PATCH 25/47] address Tejas' comment --- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 a14e70f86cf7..e21ee2da0298 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 @@ -196,9 +196,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging assert(ctx.children.size == 1) ctx.getChild(0) match { - case c if c. isInstanceOf[InsertIntoTableContext] => + case c : InsertIntoTableContext => withInsertIntoTable(c.asInstanceOf[InsertIntoTableContext], query) - case c if c.isInstanceOf[InsertOverwriteDirectoryContext] => + case c : InsertOverwriteDirectoryContext => withInsertOverwriteDirectory(c.asInstanceOf[InsertOverwriteDirectoryContext], query) } } From 392593b82cc6b7f82a426fc7a9a2976e768ff5df Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Tue, 5 Sep 2017 10:33:10 -0700 Subject: [PATCH 26/47] check point --- .../sql/catalyst/parser/AstBuilder.scala | 26 ++++-------- .../spark/sql/execution/SparkSqlParser.scala | 9 +---- .../InsertIntoDataSourceDirCommand.scala | 2 +- .../execution/InsertIntoHiveDirCommand.scala | 21 ---------- .../apache/spark/sql/hive/InsertSuite.scala | 40 +++++++++++++++++++ 5 files changed, 51 insertions(+), 47 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 e21ee2da0298..e132993c8e5f 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 @@ -196,27 +196,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging assert(ctx.children.size == 1) ctx.getChild(0) match { - case c : InsertIntoTableContext => - withInsertIntoTable(c.asInstanceOf[InsertIntoTableContext], query) - case c : InsertOverwriteDirectoryContext => - withInsertOverwriteDirectory(c.asInstanceOf[InsertOverwriteDirectoryContext], query) + case table : InsertIntoTableContext => + withInsertIntoTable(table, query) + case dir: InsertOverwriteDirContext => + val (isLocal, storage, provider) = visitInsertOverwriteDir(dir) + InsertIntoDir(isLocal, storage, provider, query, overwrite = true) + case hiveDir: InsertOverwriteHiveDirContext => + val (isLocal, storage, provider) = visitInsertOverwriteHiveDir(hiveDir) + InsertIntoDir(isLocal, storage, provider, query, overwrite = true) } } - /** - * Add an INSERT OVERWRITE [LOCAL] DIRECTORY operation to the logical plan - */ - private def withInsertOverwriteDirectory( - ctx: InsertOverwriteDirectoryContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - val (isLocal, storage, provider) = ctx match { - case dir: InsertOverwriteDirContext => visitInsertOverwriteDir(dir) - case hiveDir: InsertOverwriteHiveDirContext => visitInsertOverwriteHiveDir(hiveDir) - } - - InsertIntoDir(isLocal, storage, provider, query, overwrite = true) - } - /** * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 64a841057302..d38919b5d940 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1536,14 +1536,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { val path = Option(ctx.path).map(string).getOrElse("") - if (!path.isEmpty && storage.locationUri.isDefined) { + if (!(path.isEmpty ^ storage.locationUri.isEmpty)) { throw new ParseException( - "Directory path and 'path' in OPTIONS are both used to indicate the directory path, " + - "you can only specify one of them.", ctx) - } - if (path.isEmpty && storage.locationUri.isEmpty) { - throw new ParseException( - "You need to specify directory path or 'path' in OPTIONS, but not both", ctx) + "Directory path and 'path' in OPTIONS should be specified one, but not both", ctx) } if (!path.isEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index 3f8ed16204fa..24abfe9d827b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -63,7 +63,7 @@ case class InsertIntoDataSourceDirCommand( val isFileFormat = classOf[FileFormat].isAssignableFrom(dataSource.providingClass) if (!isFileFormat) { throw new SparkException( - "Only Data Sources providing FileFormat are supported.") + "Only Data Sources providing FileFormat are supported: " + dataSource.providingClass) } val saveMode = if (overwrite) SaveMode.Overwrite else SaveMode.ErrorIfExists diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 682a3753ea24..9e5c2d66aef9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -62,27 +62,6 @@ case class InsertIntoHiveDirCommand( assert(children.length == 1) assert(storage.locationUri.nonEmpty) -// val Array(cols, types) = children.head.output.foldLeft(Array("", "")) { case (r, a) => -// r(0) = r(0) + a.name + "," -// r(1) = r(1) + a.dataType.catalogString + ":" -// r -// } -// -// val properties = new Properties() -// properties.put("columns", cols.dropRight(1)) -// properties.put("columns.types", types.dropRight(1)) -// properties.put(serdeConstants.SERIALIZATION_LIB, -// storage.serde.getOrElse(classOf[LazySimpleSerDe].getName)) -// -// import scala.collection.JavaConverters._ -// properties.putAll(storage.properties.asJava) -// -// val tableDesc = new TableDesc( -// Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: InputFormat[_, _]]], -// Utils.classForName(storage.outputFormat.get), -// properties -// ) - val hiveTable = HiveClientImpl.toHiveTable(CatalogTable( identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")), tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index c88d6cf7cac5..5e78137892bb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -662,4 +662,44 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter assert(e.contains("Wrong FS: abc://a, expected: file:///")) } } + + test("insert overwrite to dir with mixed syntax") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + val e = intercept[ParseException] { + sql( + s""" + |INSERT OVERWRITE DIRECTORY 'file://tmp' + |USING json + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + }.getMessage + + assert(e.contains("mismatched input 'ROW'")) + } + } + + test("insert overwrite to dir with multi inserts") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + val e = intercept[ParseException] { + sql( + s""" + |INSERT OVERWRITE DIRECTORY 'file://tmp2' + |USING json + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + |INSERT OVERWRITE DIRECTORY 'file://tmp2' + |USING json + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |SELECT * FROM test_insert_table + """.stripMargin) + }.getMessage + + assert(e.contains("mismatched input 'ROW'")) + } + } } From 511cfc3f1e9d8fe57c6e3b9a8f0db63a98a44516 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Tue, 5 Sep 2017 10:40:54 -0700 Subject: [PATCH 27/47] checkpoint --- .../sql/execution/command/DDLSuite.scala | 18 ++-------- .../apache/spark/sql/hive/InsertSuite.scala | 36 ++++--------------- 2 files changed, 10 insertions(+), 44 deletions(-) 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 7164e18adb5e..ada342ca0c1c 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 @@ -2361,21 +2361,9 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { spark.sql(v1) - // use orc data source to check the data of path is right. - withTempView("orc_source") { - sql( - s""" - |CREATE TEMPORARY TABLE json_source - |USING json - |OPTIONS ( - | PATH '${dir.getCanonicalPath}' - |) - """.stripMargin) - - checkAnswer( - sql("select * from json_source"), - sql("SELECT 1 as a, 'c' as b")) - } + checkAnswer( + spark.read.json(dir.getCanonicalPath), + sql("SELECT 1 as a, 'c' as b")) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 5e78137892bb..e17e87f3c6ed 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -561,7 +561,7 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter checkAnswer( sql("select * from orc_source"), - sql("select * from src where key < 10").collect()) + sql("select * from src where key < 10")) } } } @@ -588,20 +588,9 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter """.stripMargin) // use orc data source to check the data of path is right. - withTempView("orc_source") { - sql( - s""" - |CREATE TEMPORARY VIEW orc_source - |USING org.apache.spark.sql.hive.orc - |OPTIONS ( - | PATH '${dir.getCanonicalPath}' - |) - """.stripMargin) - - checkAnswer( - sql("select * from orc_source"), - sql("select * from test_insert_table").collect()) - } + checkAnswer( + spark.read.orc(dir.getCanonicalPath), + sql("select * from test_insert_table")) } } } @@ -628,20 +617,9 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter """.stripMargin) // use orc data source to check the data of path is right. - withTempView("orc_source") { - sql( - s""" - |CREATE TEMPORARY VIEW orc_source - |USING org.apache.spark.sql.hive.orc - |OPTIONS ( - | PATH '${dir.getCanonicalPath}' - |) - """.stripMargin) - - checkAnswer( - sql("select * from orc_source"), - sql("select * from test_insert_table").collect()) - } + checkAnswer( + spark.read.orc(dir.getCanonicalPath), + sql("select * from test_insert_table")) } } } From 77948bb0050c0e2d6039ca10b05588d2e5da818f Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Tue, 5 Sep 2017 13:35:49 -0700 Subject: [PATCH 28/47] Merge insertIntoTable and insertIntoDirectory --- .../spark/sql/catalyst/parser/SqlBase.g4 | 16 ++----- .../sql/catalyst/parser/AstBuilder.scala | 43 +++++++++++++------ 2 files changed, 33 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index c4532c5e12d9..239e73ef6986 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -242,19 +242,11 @@ query : ctes? queryNoWith ; -insertIntoTable - : INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)? - | INSERT INTO TABLE? tableIdentifier partitionSpec? - ; - -insertOverwriteDirectory - : INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir - | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)? #insertOverwriteDir - ; - insertInto - : insertIntoTable - | insertOverwriteDirectory + : INSERT OVERWRITE TABLE tableIdentifier (partitionSpec (IF NOT EXISTS)?)? #insertOverwriteTable + | INSERT INTO TABLE? tableIdentifier partitionSpec? #insertIntoTable + | INSERT OVERWRITE LOCAL? DIRECTORY path=STRING rowFormat? createFileFormat? #insertOverwriteHiveDir + | INSERT OVERWRITE LOCAL? DIRECTORY (path=STRING)? tableProvider (OPTIONS options=tablePropertyList)? #insertOverwriteDir ; partitionSpecLocation 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 e132993c8e5f..70152c0bde47 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,6 +178,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging optionalMap(ctx.insertInto())(withInsertInto) } + /** + * Parameters used for writing query to a table: + * (tableIdentifier, partitionKeys, overwrite, exists). + */ + type InsertTableParams = (TableIdentifier, Map[String, Option[String]], Boolean, Boolean) + /** * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). */ @@ -193,26 +199,40 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging private def withInsertInto( ctx: InsertIntoContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { - assert(ctx.children.size == 1) - - ctx.getChild(0) match { + ctx match { case table : InsertIntoTableContext => - withInsertIntoTable(table, query) + val (tableIdent, partitionKeys, overwrite, exists) = visitInsertIntoTable(table) + InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, overwrite, exists) + case table : InsertOverwriteTableContext => + val (tableIdent, partitionKeys, overwrite, exists) = visitInsertOverwriteTable(table) + InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, overwrite, exists) case dir: InsertOverwriteDirContext => val (isLocal, storage, provider) = visitInsertOverwriteDir(dir) InsertIntoDir(isLocal, storage, provider, query, overwrite = true) case hiveDir: InsertOverwriteHiveDirContext => val (isLocal, storage, provider) = visitInsertOverwriteHiveDir(hiveDir) InsertIntoDir(isLocal, storage, provider, query, overwrite = true) + case _ => + throw new ParseException("Invalid InsertIntoContext", ctx) } } /** - * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan. + * Add an INSERT INTO TABLE operation to the logical plan. */ - private def withInsertIntoTable( - ctx: InsertIntoTableContext, - query: LogicalPlan): LogicalPlan = withOrigin(ctx) { + override def visitInsertIntoTable( + ctx: InsertIntoTableContext): InsertTableParams = withOrigin(ctx) { + val tableIdent = visitTableIdentifier(ctx.tableIdentifier) + val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) + + (tableIdent, partitionKeys, false, false) + } + + /** + * Add an INSERT OVERWRITE TABLE operation to the logical plan. + */ + override def visitInsertOverwriteTable( + ctx: InsertOverwriteTableContext): InsertTableParams = withOrigin(ctx) { val tableIdent = visitTableIdentifier(ctx.tableIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) @@ -222,12 +242,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx) } - InsertIntoTable( - UnresolvedRelation(tableIdent), - partitionKeys, - query, - ctx.OVERWRITE != null, - ctx.EXISTS != null) + (tableIdent, partitionKeys, ctx.OVERWRITE() != null, ctx.EXISTS() != null) } /** From fd9322cd17bc4eaa0daff4d06295c16aabc4cf39 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Tue, 5 Sep 2017 17:23:16 -0700 Subject: [PATCH 29/47] refactoring temp dir --- .../sql/hive/execution/HiveTmpPath.scala | 202 ++++++++++++++++++ .../execution/InsertIntoHiveDirCommand.scala | 29 ++- .../hive/execution/InsertIntoHiveTable.scala | 175 +-------------- 3 files changed, 227 insertions(+), 179 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala new file mode 100644 index 000000000000..dd84456f1ad8 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.{File, IOException} +import java.net.URI +import java.text.SimpleDateFormat +import java.util.{Date, Locale, Random} + +import scala.util.control.NonFatal +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.common.FileUtils +import org.apache.hadoop.hive.ql.exec.TaskRunner +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.hive.HiveExternalCatalog +import org.apache.spark.sql.hive.client.HiveVersion + +// Base trait for getting a temporary location for writing data +private[hive] trait HiveTmpPath extends RunnableCommand { + + var createdTempDir: Option[Path] = None + + private var stagingDir: String = "" + + def getExternalTmpPath( + sparkSession: SparkSession, + hadoopConf: Configuration, + path: Path): Path = { + import org.apache.spark.sql.hive.client.hive._ + + // Before Hive 1.1, when inserting into a table, Hive will create the staging directory under + // a common scratch directory. After the writing is finished, Hive will simply empty the table + // directory and move the staging directory to it. + // After Hive 1.1, Hive will create the staging directory under the table directory, and when + // moving staging directory to table directory, Hive will still empty the table directory, but + // will exclude the staging directory there. + // We have to follow the Hive behavior here, to avoid troubles. For example, if we create + // staging directory under the table director for Hive prior to 1.1, the staging directory will + // be removed by Hive when Hive is trying to empty the table directory. + val hiveVersionsUsingOldExternalTempPath: Set[HiveVersion] = Set(v12, v13, v14, v1_0) + val hiveVersionsUsingNewExternalTempPath: Set[HiveVersion] = Set(v1_1, v1_2, v2_0, v2_1) + + // Ensure all the supported versions are considered here. + assert(hiveVersionsUsingNewExternalTempPath ++ hiveVersionsUsingOldExternalTempPath == + allSupportedHiveVersions) + + val externalCatalog = sparkSession.sharedState.externalCatalog + val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version + stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") + val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive") + + if (hiveVersionsUsingOldExternalTempPath.contains(hiveVersion)) { + oldVersionExternalTempPath(path, hadoopConf, scratchDir) + } else if (hiveVersionsUsingNewExternalTempPath.contains(hiveVersion)) { + newVersionExternalTempPath(path, hadoopConf, stagingDir) + } else { + throw new IllegalStateException("Unsupported hive version: " + hiveVersion.fullVersion) + } + } + + def deleteExternalTmpPath(hadoopConf : Configuration) : Unit = { + // Attempt to delete the staging directory and the inclusive files. If failed, the files are + // expected to be dropped at the normal termination of VM since deleteOnExit is used. + try { + createdTempDir.foreach { path => + val fs = path.getFileSystem(hadoopConf) + if (fs.delete(path, true)) { + // If we successfully delete the staging directory, remove it from FileSystem's cache. + fs.cancelDeleteOnExit(path) + } + } + } catch { + case NonFatal(e) => + logWarning(s"Unable to delete staging directory: $stagingDir.\n" + e) + } + } + + // Mostly copied from Context.java#getExternalTmpPath of Hive 0.13 + private def oldVersionExternalTempPath( + path: Path, + hadoopConf: Configuration, + scratchDir: String): Path = { + val extURI: URI = path.toUri + val scratchPath = new Path(scratchDir, executionId) + var dirPath = new Path( + extURI.getScheme, + extURI.getAuthority, + scratchPath.toUri.getPath + "-" + TaskRunner.getTaskRunnerID()) + + try { + val fs: FileSystem = dirPath.getFileSystem(hadoopConf) + dirPath = new Path(fs.makeQualified(dirPath).toString()) + + if (!FileUtils.mkdir(fs, dirPath, true, hadoopConf)) { + throw new IllegalStateException("Cannot create staging directory: " + dirPath.toString) + } + createdTempDir = Some(dirPath) + fs.deleteOnExit(dirPath) + } catch { + case e: IOException => + throw new RuntimeException("Cannot create staging directory: " + dirPath.toString, e) + } + dirPath + } + + // Mostly copied from Context.java#getExternalTmpPath of Hive 1.2 + private def newVersionExternalTempPath( + path: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { + val extURI: URI = path.toUri + if (extURI.getScheme == "viewfs") { + getExtTmpPathRelTo(path.getParent, hadoopConf, stagingDir) + } else { + new Path(getExternalScratchDir(extURI, hadoopConf, stagingDir), "-ext-10000") + } + } + + private def getExtTmpPathRelTo( + path: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { + new Path(getStagingDir(path, hadoopConf, stagingDir), "-ext-10000") // Hive uses 10000 + } + + private def getExternalScratchDir( + extURI: URI, + hadoopConf: Configuration, + stagingDir: String): Path = { + getStagingDir( + new Path(extURI.getScheme, extURI.getAuthority, extURI.getPath), + hadoopConf, + stagingDir) + } + + private def getStagingDir( + inputPath: Path, + hadoopConf: Configuration, + stagingDir: String): Path = { + val inputPathUri: URI = inputPath.toUri + val inputPathName: String = inputPathUri.getPath + val fs: FileSystem = inputPath.getFileSystem(hadoopConf) + var stagingPathName: String = + if (inputPathName.indexOf(stagingDir) == -1) { + new Path(inputPathName, stagingDir).toString + } else { + inputPathName.substring(0, inputPathName.indexOf(stagingDir) + stagingDir.length) + } + + // SPARK-20594: This is a walk-around fix to resolve a Hive bug. Hive requires that the + // staging directory needs to avoid being deleted when users set hive.exec.stagingdir + // under the table directory. + if (FileUtils.isSubDir(new Path(stagingPathName), inputPath, fs) && + !stagingPathName.stripPrefix(inputPathName).stripPrefix(File.separator).startsWith(".")) { + logDebug(s"The staging dir '$stagingPathName' should be a child directory starts " + + "with '.' to avoid being deleted if we set hive.exec.stagingdir under the table " + + "directory.") + stagingPathName = new Path(inputPathName, ".hive-staging").toString + } + + val dir: Path = + fs.makeQualified( + new Path(stagingPathName + "_" + executionId + "-" + TaskRunner.getTaskRunnerID)) + logDebug("Created staging dir = " + dir + " for path = " + inputPath) + try { + if (!FileUtils.mkdir(fs, dir, true, hadoopConf)) { + throw new IllegalStateException("Cannot create staging directory '" + dir.toString + "'") + } + createdTempDir = Some(dir) + fs.deleteOnExit(dir) + } catch { + case e: IOException => + throw new RuntimeException( + "Cannot create staging directory '" + dir.toString + "': " + e.getMessage, e) + } + dir + } + + private def executionId: String = { + val rand: Random = new Random + val format = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS", Locale.US) + "hive_" + format.format(new Date) + "_" + Math.abs(rand.nextLong) + } +} + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 9e5c2d66aef9..64570be99e11 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -18,19 +18,19 @@ package org.apache.spark.sql.hive.execution import scala.language.existentials - import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.mapred._ - +import org.apache.spark.SparkException import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.client.HiveClientImpl /** @@ -54,7 +54,7 @@ case class InsertIntoHiveDirCommand( isLocal: Boolean, storage: CatalogStorageFormat, query: LogicalPlan, - overwrite: Boolean) extends SaveAsHiveFile { + overwrite: Boolean) extends SaveAsHiveFile with HiveTmpPath { override def children: Seq[LogicalPlan] = query :: Nil @@ -108,15 +108,26 @@ case class InsertIntoHiveDirCommand( qualifiedPath } + val tmpLocation = getExternalTmpPath(sparkSession, hadoopConf, writeToPath) val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc( writeToPath.toString, tableDesc, false) - saveAsHiveFile( - sparkSession = sparkSession, - plan = children.head, - hadoopConf = hadoopConf, - fileSinkConf = fileSinkConf, - outputLocation = targetPath.toString) + try { + saveAsHiveFile( + sparkSession = sparkSession, + plan = children.head, + hadoopConf = hadoopConf, + fileSinkConf = fileSinkConf, + outputLocation = writeToPath.toString) + + // val fs = writeToPath.getFileSystem(hadoopConf) + // fs.rename(tmpLocation, writeToPath) + // deleteExternalTmpPath(hadoopConf) + + } catch { + case e => + throw new SparkException("Failed inserting overwrite directory " + storage.locationUri.get) + } Seq.empty[Row] } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 112946b510af..5bdc97a2982d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -17,18 +17,10 @@ package org.apache.spark.sql.hive.execution -import java.io.{File, IOException} -import java.net.URI -import java.text.SimpleDateFormat -import java.util.{Date, Locale, Random} - import scala.util.control.NonFatal -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.hive.common.FileUtils +import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.ErrorMsg -import org.apache.hadoop.hive.ql.exec.TaskRunner import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.spark.SparkException @@ -40,7 +32,7 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.CommandUtils import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} -import org.apache.spark.sql.hive.client.{HiveClientImpl, HiveVersion} +import org.apache.spark.sql.hive.client.HiveClientImpl /** @@ -78,152 +70,10 @@ case class InsertIntoHiveTable( partition: Map[String, Option[String]], query: LogicalPlan, overwrite: Boolean, - ifPartitionNotExists: Boolean) extends SaveAsHiveFile { + ifPartitionNotExists: Boolean) extends SaveAsHiveFile with HiveTmpPath { override def children: Seq[LogicalPlan] = query :: Nil - var createdTempDir: Option[Path] = None - - private def executionId: String = { - val rand: Random = new Random - val format = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS", Locale.US) - "hive_" + format.format(new Date) + "_" + Math.abs(rand.nextLong) - } - - private def getStagingDir( - inputPath: Path, - hadoopConf: Configuration, - stagingDir: String): Path = { - val inputPathUri: URI = inputPath.toUri - val inputPathName: String = inputPathUri.getPath - val fs: FileSystem = inputPath.getFileSystem(hadoopConf) - var stagingPathName: String = - if (inputPathName.indexOf(stagingDir) == -1) { - new Path(inputPathName, stagingDir).toString - } else { - inputPathName.substring(0, inputPathName.indexOf(stagingDir) + stagingDir.length) - } - - // SPARK-20594: This is a walk-around fix to resolve a Hive bug. Hive requires that the - // staging directory needs to avoid being deleted when users set hive.exec.stagingdir - // under the table directory. - if (FileUtils.isSubDir(new Path(stagingPathName), inputPath, fs) && - !stagingPathName.stripPrefix(inputPathName).stripPrefix(File.separator).startsWith(".")) { - logDebug(s"The staging dir '$stagingPathName' should be a child directory starts " + - "with '.' to avoid being deleted if we set hive.exec.stagingdir under the table " + - "directory.") - stagingPathName = new Path(inputPathName, ".hive-staging").toString - } - - val dir: Path = - fs.makeQualified( - new Path(stagingPathName + "_" + executionId + "-" + TaskRunner.getTaskRunnerID)) - logDebug("Created staging dir = " + dir + " for path = " + inputPath) - try { - if (!FileUtils.mkdir(fs, dir, true, hadoopConf)) { - throw new IllegalStateException("Cannot create staging directory '" + dir.toString + "'") - } - createdTempDir = Some(dir) - fs.deleteOnExit(dir) - } catch { - case e: IOException => - throw new RuntimeException( - "Cannot create staging directory '" + dir.toString + "': " + e.getMessage, e) - } - dir - } - - private def getExternalScratchDir( - extURI: URI, - hadoopConf: Configuration, - stagingDir: String): Path = { - getStagingDir( - new Path(extURI.getScheme, extURI.getAuthority, extURI.getPath), - hadoopConf, - stagingDir) - } - - def getExternalTmpPath( - path: Path, - hiveVersion: HiveVersion, - hadoopConf: Configuration, - stagingDir: String, - scratchDir: String): Path = { - import org.apache.spark.sql.hive.client.hive._ - - // Before Hive 1.1, when inserting into a table, Hive will create the staging directory under - // a common scratch directory. After the writing is finished, Hive will simply empty the table - // directory and move the staging directory to it. - // After Hive 1.1, Hive will create the staging directory under the table directory, and when - // moving staging directory to table directory, Hive will still empty the table directory, but - // will exclude the staging directory there. - // We have to follow the Hive behavior here, to avoid troubles. For example, if we create - // staging directory under the table director for Hive prior to 1.1, the staging directory will - // be removed by Hive when Hive is trying to empty the table directory. - val hiveVersionsUsingOldExternalTempPath: Set[HiveVersion] = Set(v12, v13, v14, v1_0) - val hiveVersionsUsingNewExternalTempPath: Set[HiveVersion] = Set(v1_1, v1_2, v2_0, v2_1) - - // Ensure all the supported versions are considered here. - assert(hiveVersionsUsingNewExternalTempPath ++ hiveVersionsUsingOldExternalTempPath == - allSupportedHiveVersions) - - if (hiveVersionsUsingOldExternalTempPath.contains(hiveVersion)) { - oldVersionExternalTempPath(path, hadoopConf, scratchDir) - } else if (hiveVersionsUsingNewExternalTempPath.contains(hiveVersion)) { - newVersionExternalTempPath(path, hadoopConf, stagingDir) - } else { - throw new IllegalStateException("Unsupported hive version: " + hiveVersion.fullVersion) - } - } - - // Mostly copied from Context.java#getExternalTmpPath of Hive 0.13 - def oldVersionExternalTempPath( - path: Path, - hadoopConf: Configuration, - scratchDir: String): Path = { - val extURI: URI = path.toUri - val scratchPath = new Path(scratchDir, executionId) - var dirPath = new Path( - extURI.getScheme, - extURI.getAuthority, - scratchPath.toUri.getPath + "-" + TaskRunner.getTaskRunnerID()) - - try { - val fs: FileSystem = dirPath.getFileSystem(hadoopConf) - dirPath = new Path(fs.makeQualified(dirPath).toString()) - - if (!FileUtils.mkdir(fs, dirPath, true, hadoopConf)) { - throw new IllegalStateException("Cannot create staging directory: " + dirPath.toString) - } - createdTempDir = Some(dirPath) - fs.deleteOnExit(dirPath) - } catch { - case e: IOException => - throw new RuntimeException("Cannot create staging directory: " + dirPath.toString, e) - } - dirPath - } - - // Mostly copied from Context.java#getExternalTmpPath of Hive 1.2 - def newVersionExternalTempPath( - path: Path, - hadoopConf: Configuration, - stagingDir: String): Path = { - val extURI: URI = path.toUri - if (extURI.getScheme == "viewfs") { - getExtTmpPathRelTo(path.getParent, hadoopConf, stagingDir) - } else { - new Path(getExternalScratchDir(extURI, hadoopConf, stagingDir), "-ext-10000") - } - } - - def getExtTmpPathRelTo( - path: Path, - hadoopConf: Configuration, - stagingDir: String): Path = { - new Path(getStagingDir(path, hadoopConf, stagingDir), "-ext-10000") // Hive uses 10000 - } - /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the @@ -233,10 +83,7 @@ case class InsertIntoHiveTable( assert(children.length == 1) val externalCatalog = sparkSession.sharedState.externalCatalog - val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version val hadoopConf = sparkSession.sessionState.newHadoopConf() - val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") - val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive") val hiveQlTable = HiveClientImpl.toHiveTable(table) // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer @@ -251,8 +98,7 @@ case class InsertIntoHiveTable( hiveQlTable.getMetadata ) val tableLocation = hiveQlTable.getDataLocation - val tmpLocation = - getExternalTmpPath(tableLocation, hiveVersion, hadoopConf, stagingDir, scratchDir) + val tmpLocation = getExternalTmpPath(sparkSession, hadoopConf, tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) val numDynamicPartitions = partition.values.count(_.isEmpty) @@ -396,18 +242,7 @@ case class InsertIntoHiveTable( // Attempt to delete the staging directory and the inclusive files. If failed, the files are // expected to be dropped at the normal termination of VM since deleteOnExit is used. - try { - createdTempDir.foreach { path => - val fs = path.getFileSystem(hadoopConf) - if (fs.delete(path, true)) { - // If we successfully delete the staging directory, remove it from FileSystem's cache. - fs.cancelDeleteOnExit(path) - } - } - } catch { - case NonFatal(e) => - logWarning(s"Unable to delete staging directory: $stagingDir.\n" + e) - } + deleteExternalTmpPath(hadoopConf) // un-cache this table. sparkSession.catalog.uncacheTable(table.identifier.quotedString) From e590847a1183095ecc23f513fd091bff0e7d6a8c Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Tue, 5 Sep 2017 23:21:16 -0700 Subject: [PATCH 30/47] add TODO for tmpPath --- .../sql/hive/execution/InsertIntoHiveDirCommand.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 64570be99e11..aa9adbe6caa8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -108,11 +108,13 @@ case class InsertIntoHiveDirCommand( qualifiedPath } - val tmpLocation = getExternalTmpPath(sparkSession, hadoopConf, writeToPath) + val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath) + // TODO: using tmpPath val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc( writeToPath.toString, tableDesc, false) try { + // TODO: using tmpPath saveAsHiveFile( sparkSession = sparkSession, plan = children.head, @@ -120,9 +122,9 @@ case class InsertIntoHiveDirCommand( fileSinkConf = fileSinkConf, outputLocation = writeToPath.toString) - // val fs = writeToPath.getFileSystem(hadoopConf) - // fs.rename(tmpLocation, writeToPath) - // deleteExternalTmpPath(hadoopConf) + // TODO: move files from tmpPath to writeToPath + + deleteExternalTmpPath(hadoopConf) } catch { case e => From dd6a418c3bcbae82517b044a8c9d423c375cf4e5 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 00:24:53 -0700 Subject: [PATCH 31/47] use tmpPath --- .../execution/InsertIntoHiveDirCommand.scala | 45 +++++++++---------- 1 file changed, 22 insertions(+), 23 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index aa9adbe6caa8..8175a7438d07 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -18,19 +18,20 @@ package org.apache.spark.sql.hive.execution import scala.language.existentials + import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.mapred._ + import org.apache.spark.SparkException import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.client.HiveClientImpl /** @@ -84,48 +85,46 @@ case class InsertIntoHiveDirCommand( val writeToPath = if (isLocal) { val localFileSystem = FileSystem.getLocal(jobConf) - val localPath = localFileSystem.makeQualified(targetPath) - if (localFileSystem.exists(localPath)) { - if (overwrite) { - localFileSystem.delete(localPath, true) - } else { - throw new RuntimeException("Directory '" + localPath.toString + "' already exists") - } - } - localPath + localFileSystem.makeQualified(targetPath) } else { val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf) val dfs = qualifiedPath.getFileSystem(jobConf) - if (dfs.exists(qualifiedPath)) { - if (overwrite) { - dfs.delete(qualifiedPath, true) - } else { - throw new RuntimeException("Directory '" + qualifiedPath.toString + "' already exists") - } - } else { + if (!dfs.exists(qualifiedPath)) { dfs.mkdirs(qualifiedPath.getParent) } qualifiedPath } val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath) - // TODO: using tmpPath val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc( - writeToPath.toString, tableDesc, false) + tmpPath.toString, tableDesc, false) try { - // TODO: using tmpPath saveAsHiveFile( sparkSession = sparkSession, plan = children.head, hadoopConf = hadoopConf, fileSinkConf = fileSinkConf, - outputLocation = writeToPath.toString) + outputLocation = tmpPath.toString) + + val fs = writeToPath.getFileSystem(hadoopConf) + if (overwrite) { + val existFiles = fs.listStatus(writeToPath) + existFiles.foreach { + existFile => + if (existFile.getPath != createdTempDir.get) { + fs.delete(existFile.getPath, true) + } + } + } - // TODO: move files from tmpPath to writeToPath + val tmpFiles = fs.listStatus(tmpPath) + tmpFiles.foreach { + tmpFile => + fs.rename(tmpFile.getPath, writeToPath) + } deleteExternalTmpPath(hadoopConf) - } catch { case e => throw new SparkException("Failed inserting overwrite directory " + storage.locationUri.get) From c2c693c3414037697e5c9d4afd4433688c6f199f Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 00:27:28 -0700 Subject: [PATCH 32/47] fix style --- .../scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala index dd84456f1ad8..f2b04c81fffc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala @@ -23,10 +23,12 @@ import java.text.SimpleDateFormat import java.util.{Date, Locale, Random} import scala.util.control.NonFatal + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.exec.TaskRunner + import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.hive.HiveExternalCatalog From e9c88b5ee0629f55abf90b7d6b75b07b442eb396 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 00:41:13 -0700 Subject: [PATCH 33/47] add exists check --- .../spark/sql/hive/execution/InsertIntoHiveDirCommand.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 8175a7438d07..8e56db16551a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -108,7 +108,7 @@ case class InsertIntoHiveDirCommand( outputLocation = tmpPath.toString) val fs = writeToPath.getFileSystem(hadoopConf) - if (overwrite) { + if (overwrite && fs.exists(writeToPath)) { val existFiles = fs.listStatus(writeToPath) existFiles.foreach { existFile => From b64520b9c09842e6834ae595407f354c22955e43 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 08:49:34 -0700 Subject: [PATCH 34/47] fix build --- .../spark/sql/hive/execution/InsertIntoHiveDirCommand.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 8e56db16551a..dce6a42d439d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -126,8 +126,9 @@ case class InsertIntoHiveDirCommand( deleteExternalTmpPath(hadoopConf) } catch { - case e => - throw new SparkException("Failed inserting overwrite directory " + storage.locationUri.get) + case e : Throwable => + throw new SparkException( + "Failed inserting overwrite directory " + storage.locationUri.get, e) } Seq.empty[Row] From 3aaf6e83eaa8dff387cd9cd2c5f2f61f94bbcca3 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 10:51:00 -0700 Subject: [PATCH 35/47] fix build failure --- .../org/apache/spark/sql/execution/command/DDLParserSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 4749d2d863f1..08b405b312dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -565,7 +565,7 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { parser.parsePlan(v4) } assert(e4.message.contains( - "Directory path and 'path' in OPTIONS are both used to indicate the directory path")) + "Directory path and 'path' in OPTIONS should be specified one, but not both")) } // ALTER TABLE table_name RENAME TO new_table_name; From b461e00e425660e33fdbc24a75884a2a2e2da4b8 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 10:54:44 -0700 Subject: [PATCH 36/47] fix build failure --- .../apache/spark/sql/execution/command/DDLParserSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 08b405b312dd..49b5a09817a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -539,7 +539,8 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { val e2 = intercept[ParseException] { parser.parsePlan(v2) } - assert(e2.message.contains("You need to specify directory path or 'path' in OPTIONS")) + assert(e2.message.contains( + "Directory path and 'path' in OPTIONS should be specified one, but not both")) val v3 = """ From e9c24deced902a207ad32a0a32011c31e193761d Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 15:19:24 -0700 Subject: [PATCH 37/47] address's gatorsmile's comment --- .../spark/sql/catalyst/parser/AstBuilder.scala | 13 ++++++++----- .../plans/logical/basicLogicalOperators.scala | 4 ++++ .../spark/sql/hive/execution/HiveTmpPath.scala | 10 ++++------ .../hive/execution/InsertIntoHiveDirCommand.scala | 3 ++- .../sql/hive/execution/InsertIntoHiveTable.scala | 3 ++- .../spark/sql/hive/execution/SaveAsHiveFile.scala | 2 -- 6 files changed, 20 insertions(+), 15 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 70152c0bde47..9c5ca54ddc3a 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 @@ -191,19 +191,22 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging /** * Add an - * INSERT INTO [TABLE] or - * INSERT OVERWRITE TABLE or - * INSERT OVERWRITE [LOCAL] DIRECTORY + * {{{ + * INSERT OVERWRITE TABLE tableIdentifier [partitionSpec [IF NOT EXISTS]]? + * INSERT INTO [TABLE] tableIdentifier [partitionSpec] + * INSERT OVERWRITE [LOCAL] DIRECTORY STRING [rowFormat] [createFileFormat] + * INSERT OVERWRITE [LOCAL] DIRECTORY [STRING] tableProvider [OPTIONS tablePropertyList] + * }}} * operation to logical plan */ private def withInsertInto( ctx: InsertIntoContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { ctx match { - case table : InsertIntoTableContext => + case table: InsertIntoTableContext => val (tableIdent, partitionKeys, overwrite, exists) = visitInsertIntoTable(table) InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, overwrite, exists) - case table : InsertOverwriteTableContext => + case table: InsertOverwriteTableContext => val (tableIdent, partitionKeys, overwrite, exists) = visitInsertOverwriteTable(table) InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, overwrite, exists) case dir: InsertOverwriteDirContext => 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 9a5e7616aad4..5eca7aa48411 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 @@ -367,6 +367,9 @@ case class InsertIntoTable( * @param provider Specifies what data source to use; only used for data source file. * @param child The query to be executed * @param overwrite If true, the existing directory will be overwritten + * + * Note that this plan is unresolved and has to be replaced by the concrete implementations + * during analysis. */ case class InsertIntoDir( isLocal: Boolean, @@ -378,6 +381,7 @@ case class InsertIntoDir( override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = Seq.empty + override lazy val resolved: Boolean = false } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala index f2b04c81fffc..49d91c26653b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala @@ -29,18 +29,16 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.exec.TaskRunner +import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.client.HiveVersion // Base trait for getting a temporary location for writing data -private[hive] trait HiveTmpPath extends RunnableCommand { +private[hive] trait HiveTmpPath extends Logging { var createdTempDir: Option[Path] = None - private var stagingDir: String = "" - def getExternalTmpPath( sparkSession: SparkSession, hadoopConf: Configuration, @@ -65,7 +63,7 @@ private[hive] trait HiveTmpPath extends RunnableCommand { val externalCatalog = sparkSession.sharedState.externalCatalog val hiveVersion = externalCatalog.asInstanceOf[HiveExternalCatalog].client.version - stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") + val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") val scratchDir = hadoopConf.get("hive.exec.scratchdir", "/tmp/hive") if (hiveVersionsUsingOldExternalTempPath.contains(hiveVersion)) { @@ -77,7 +75,7 @@ private[hive] trait HiveTmpPath extends RunnableCommand { } } - def deleteExternalTmpPath(hadoopConf : Configuration) : Unit = { + def deleteExternalTmpPath(hadoopConf: Configuration, stagingDir: String) : Unit = { // Attempt to delete the staging directory and the inclusive files. If failed, the files are // expected to be dropped at the normal termination of VM since deleteOnExit is used. try { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index dce6a42d439d..804ebc018fef 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -124,7 +124,8 @@ case class InsertIntoHiveDirCommand( fs.rename(tmpFile.getPath, writeToPath) } - deleteExternalTmpPath(hadoopConf) + val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") + deleteExternalTmpPath(hadoopConf, stagingDir) } catch { case e : Throwable => throw new SparkException( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 5bdc97a2982d..807fb5dfcd78 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -84,6 +84,7 @@ case class InsertIntoHiveTable( val externalCatalog = sparkSession.sharedState.externalCatalog val hadoopConf = sparkSession.sessionState.newHadoopConf() + val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") val hiveQlTable = HiveClientImpl.toHiveTable(table) // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer @@ -242,7 +243,7 @@ case class InsertIntoHiveTable( // Attempt to delete the staging directory and the inclusive files. If failed, the files are // expected to be dropped at the normal termination of VM since deleteOnExit is used. - deleteExternalTmpPath(hadoopConf) + deleteExternalTmpPath(hadoopConf, stagingDir) // un-cache this table. sparkSession.catalog.uncacheTable(table.identifier.quotedString) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index f3927952529e..d7992cdf9007 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -41,8 +41,6 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { bucketSpec: Option[BucketSpec] = None, options: Map[String, String] = Map.empty): Unit = { - val sessionState = sparkSession.sessionState - val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean if (isCompressed) { // Please note that isCompressed, "mapreduce.output.fileoutputformat.compress", From 0c03a2be092ec2336ee053fe14fd4ccf80f8b7b5 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 15:34:05 -0700 Subject: [PATCH 38/47] address gatorsmile's comment --- .../sql/execution/command/DDLSuite.scala | 39 ------------------ .../spark/sql/sources/InsertSuite.scala | 40 +++++++++++++++++++ .../sql/hive/execution/SaveAsHiveFile.scala | 9 ++--- 3 files changed, 43 insertions(+), 45 deletions(-) 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 ada342ca0c1c..7fdcce4f017e 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 @@ -2347,45 +2347,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("insert overwrite directory") { - withTempDir { dir => - val path = dir.toURI.getPath - - val v1 = - s""" - | INSERT OVERWRITE DIRECTORY '${path}' - | USING json - | OPTIONS (a 1, b 0.1, c TRUE) - | SELECT 1 as a, 'c' as b - """.stripMargin - - spark.sql(v1) - - checkAnswer( - spark.read.json(dir.getCanonicalPath), - sql("SELECT 1 as a, 'c' as b")) - } - } - - test("insert overwrite directory to data source not providing FileFormat") { - withTempDir { dir => - val path = dir.toURI.getPath - - val v1 = - s""" - | INSERT OVERWRITE DIRECTORY '${path}' - | USING JDBC - | OPTIONS (a 1, b 0.1, c TRUE) - | SELECT 1 as a, 'c' as b - """.stripMargin - val e = intercept[SparkException] { - spark.sql(v1) - }.getMessage - - assert(e.contains("Only Data Sources providing FileFormat are supported")) - } - } - Seq(true, false).foreach { caseSensitive => test(s"alter table add columns with existing column name - caseSensitive $caseSensitive") { withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitive") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 41abff2a5da2..de5795e131b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import java.io.File +import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils @@ -366,4 +367,43 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { Row(Array(1, 2), Array("a", "b"))) } } + + test("insert overwrite directory") { + withTempDir { dir => + val path = dir.toURI.getPath + + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY '${path}' + | USING json + | OPTIONS (a 1, b 0.1, c TRUE) + | SELECT 1 as a, 'c' as b + """.stripMargin + + spark.sql(v1) + + checkAnswer( + spark.read.json(dir.getCanonicalPath), + sql("SELECT 1 as a, 'c' as b")) + } + } + + test("insert overwrite directory to data source not providing FileFormat") { + withTempDir { dir => + val path = dir.toURI.getPath + + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY '${path}' + | USING JDBC + | OPTIONS (a 1, b 0.1, c TRUE) + | SELECT 1 as a, 'c' as b + """.stripMargin + val e = intercept[SparkException] { + spark.sql(v1) + }.getMessage + + assert(e.contains("Only Data Sources providing FileFormat are supported")) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index d7992cdf9007..7de9b421245f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -21,7 +21,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.DataWritingCommand @@ -37,9 +36,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { hadoopConf: Configuration, fileSinkConf: FileSinkDesc, outputLocation: String, - partitionAttributes: Seq[Attribute] = Nil, - bucketSpec: Option[BucketSpec] = None, - options: Map[String, String] = Map.empty): Unit = { + partitionAttributes: Seq[Attribute] = Nil): Unit = { val isCompressed = hadoopConf.get("hive.exec.compress.output", "false").toBoolean if (isCompressed) { @@ -68,9 +65,9 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { outputSpec = FileFormatWriter.OutputSpec(outputLocation, Map.empty), hadoopConf = hadoopConf, partitionColumns = partitionAttributes, - bucketSpec = bucketSpec, + bucketSpec = None, statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), - options = options) + options = Map.empty) } } From 6c24b1be90fdf0e65c80ae24f81c75d34f7e1542 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Wed, 6 Sep 2017 15:35:18 -0700 Subject: [PATCH 39/47] remove unused import --- .../scala/org/apache/spark/sql/execution/command/DDLSuite.scala | 1 - 1 file changed, 1 deletion(-) 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 7fdcce4f017e..ad6fc20df1f0 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 @@ -24,7 +24,6 @@ import java.util.Locale import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach -import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} From 0ec103e8f8a91a8ba418fa84bda9965155c51f3f Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 7 Sep 2017 11:00:21 -0700 Subject: [PATCH 40/47] address gatorsmile's comments --- .../InsertIntoDataSourceDirCommand.scala | 7 +++-- .../spark/sql/execution/command/ddl.scala | 31 +++++++++---------- .../datasources/DataSourceStrategy.scala | 2 +- .../execution/command/DDLParserSuite.scala | 15 ++++++--- .../spark/sql/sources/InsertSuite.scala | 3 ++ .../sql/hive/execution/HiveTmpPath.scala | 3 +- .../execution/InsertIntoHiveDirCommand.scala | 19 ++++-------- .../hive/execution/InsertIntoHiveTable.scala | 3 +- 8 files changed, 42 insertions(+), 41 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index 24abfe9d827b..971c6720e1da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.SparkPlan /** * A command used to write the result of a query to a directory. @@ -44,10 +45,10 @@ case class InsertIntoDataSourceDirCommand( query: LogicalPlan, overwrite: Boolean) extends RunnableCommand { - override def innerChildren: Seq[LogicalPlan] = Seq(query) + override def children: Seq[LogicalPlan] = Seq(query) - override def run(sparkSession: SparkSession): Seq[Row] = { - assert(innerChildren.length == 1) + override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { + assert(children.length == 1) assert(storage.locationUri.nonEmpty, "Directory path is required") assert(!provider.isEmpty, "Data source is required") 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 a868bf85e875..b06f4ccaa3bb 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 @@ -34,7 +34,7 @@ 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.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.datasources.PartitioningUtils +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitioningUtils} import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter import org.apache.spark.sql.internal.HiveSerDe @@ -853,20 +853,6 @@ object DDLUtils { } } -<<<<<<< HEAD - - /** - * Throws exception if outputPath tries to overwrite inputpath. - */ - def verifyNotReadPath(query: LogicalPlan, outputPath: Path) : Unit = { - val inputPaths = query.collect { - case LogicalRelation(r: HadoopFsRelation, _, _, _) => r.location.rootPaths - }.flatten - - if (inputPaths.contains(outputPath)) { - throw new AnalysisException( - "Cannot overwrite a path that is also being read from.") -======= private[sql] def checkDataSchemaFieldNames(table: CatalogTable): Unit = { table.provider.foreach { _.toLowerCase(Locale.ROOT) match { @@ -882,7 +868,20 @@ object DDLUtils { case "orc" => OrcFileFormat.checkFieldNames(table.dataSchema) case _ => } ->>>>>>> master + } + } + + /** + * Throws exception if outputPath tries to overwrite inputpath. + */ + def verifyNotReadPath(query: LogicalPlan, outputPath: Path) : Unit = { + val inputPaths = query.collect { + case LogicalRelation(r: HadoopFsRelation, _, _, _) => r.location.rootPaths + }.flatten + + if (inputPaths.contains(outputPath)) { + throw new AnalysisException( + "Cannot overwrite a path that is also being read from.") } } } 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 5bdf22dee07e..018f24e290b4 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 @@ -145,7 +145,7 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast parts, query, overwrite, false) if parts.isEmpty => InsertIntoDataSourceCommand(l, query, overwrite) - case InsertIntoDir(isLocal, storage, provider, query, overwrite) + case InsertIntoDir(_, storage, provider, query, overwrite) if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) != DDLUtils.HIVE_PROVIDER => val outputPath = new Path(storage.locationUri.get) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 49b5a09817a3..0375e58dfdc6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -32,8 +32,8 @@ import org.apache.spark.sql.catalyst.dsl.plans.DslLogicalPlan import org.apache.spark.sql.catalyst.expressions.JsonTuple import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan, - Project, ScriptTransformation} +import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{Project, ScriptTransformation} import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -529,10 +529,10 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a" parser.parsePlan(v1) match { case InsertIntoDir(_, storage, provider, query, overwrite) => - assert(storage.locationUri != None && storage.locationUri.get.toString == "/tmp/file") + assert(storage.locationUri.isDefined && storage.locationUri.get.toString == "/tmp/file") case other => fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + - " from query," + s"got ${other.getClass.getName}: $v1") + " from query," + s" got ${other.getClass.getName}: $v1") } val v2 = "INSERT OVERWRITE DIRECTORY USING parquet SELECT 1 as a" @@ -550,7 +550,12 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { """.stripMargin parser.parsePlan(v3) match { case InsertIntoDir(_, storage, provider, query, overwrite) => - assert(storage.locationUri != None && provider == Some("json")) + assert(storage.locationUri.isDefined && provider == Some("json")) + assert(storage.properties.get("path") == Some("/tmp/file")) + assert(storage.properties.get("a") == Some("1")) + assert(storage.properties.get("b") == Some("0.1")) + assert(storage.properties.get("c") == Some("TRUE")) + assert(!storage.properties.contains("abc")) case other => fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + " from query," + s"got ${other.getClass.getName}: $v1") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index de5795e131b9..0bab8bc5782e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -368,6 +368,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { } } + // Jane test("insert overwrite directory") { withTempDir { dir => val path = dir.toURI.getPath @@ -406,4 +407,6 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { assert(e.contains("Only Data Sources providing FileFormat are supported")) } } + + } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala index 49d91c26653b..15ca1dfc76d1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTmpPath.scala @@ -75,7 +75,7 @@ private[hive] trait HiveTmpPath extends Logging { } } - def deleteExternalTmpPath(hadoopConf: Configuration, stagingDir: String) : Unit = { + def deleteExternalTmpPath(hadoopConf: Configuration) : Unit = { // Attempt to delete the staging directory and the inclusive files. If failed, the files are // expected to be dropped at the normal termination of VM since deleteOnExit is used. try { @@ -88,6 +88,7 @@ private[hive] trait HiveTmpPath extends Logging { } } catch { case NonFatal(e) => + val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") logWarning(s"Unable to delete staging directory: $stagingDir.\n" + e) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 804ebc018fef..02a5508ee47e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -109,25 +109,18 @@ case class InsertIntoHiveDirCommand( val fs = writeToPath.getFileSystem(hadoopConf) if (overwrite && fs.exists(writeToPath)) { - val existFiles = fs.listStatus(writeToPath) - existFiles.foreach { - existFile => - if (existFile.getPath != createdTempDir.get) { - fs.delete(existFile.getPath, true) - } + fs.listStatus(writeToPath).foreach { existFile => + if (Option(existFile.getPath) != createdTempDir) fs.delete(existFile.getPath, true) } } - val tmpFiles = fs.listStatus(tmpPath) - tmpFiles.foreach { - tmpFile => - fs.rename(tmpFile.getPath, writeToPath) + fs.listStatus(tmpPath).foreach { + tmpFile => fs.rename(tmpFile.getPath, writeToPath) } - val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") - deleteExternalTmpPath(hadoopConf, stagingDir) + deleteExternalTmpPath(hadoopConf) } catch { - case e : Throwable => + case e: Throwable => throw new SparkException( "Failed inserting overwrite directory " + storage.locationUri.get, e) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 807fb5dfcd78..5bdc97a2982d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -84,7 +84,6 @@ case class InsertIntoHiveTable( val externalCatalog = sparkSession.sharedState.externalCatalog val hadoopConf = sparkSession.sessionState.newHadoopConf() - val stagingDir = hadoopConf.get("hive.exec.stagingdir", ".hive-staging") val hiveQlTable = HiveClientImpl.toHiveTable(table) // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer @@ -243,7 +242,7 @@ case class InsertIntoHiveTable( // Attempt to delete the staging directory and the inclusive files. If failed, the files are // expected to be dropped at the normal termination of VM since deleteOnExit is used. - deleteExternalTmpPath(hadoopConf, stagingDir) + deleteExternalTmpPath(hadoopConf) // un-cache this table. sparkSession.catalog.uncacheTable(table.identifier.quotedString) From 160c0ec1b645b25c0cf0c423c2ad0d123a7c61f7 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 7 Sep 2017 11:02:21 -0700 Subject: [PATCH 41/47] fix style --- .../sql/execution/command/InsertIntoDataSourceDirCommand.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index 971c6720e1da..f0d8e6ddb65f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -21,8 +21,8 @@ import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources._ /** * A command used to write the result of a query to a directory. From 95ebfd3e794ba560da0ab0be76976bfccf6f5e24 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 7 Sep 2017 11:54:27 -0700 Subject: [PATCH 42/47] add more unittest --- .../spark/sql/sources/InsertSuite.scala | 23 ++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 0bab8bc5782e..875b74551add 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -368,7 +368,6 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { } } - // Jane test("insert overwrite directory") { withTempDir { dir => val path = dir.toURI.getPath @@ -389,6 +388,26 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { } } + test("insert overwrite directory with path in options") { + withTempDir { dir => + val path = dir.toURI.getPath + + val v1 = + s""" + | INSERT OVERWRITE DIRECTORY + | USING json + | OPTIONS ('path' '${path}') + | SELECT 1 as a, 'c' as b + """.stripMargin + + spark.sql(v1) + + checkAnswer( + spark.read.json(dir.getCanonicalPath), + sql("SELECT 1 as a, 'c' as b")) + } + } + test("insert overwrite directory to data source not providing FileFormat") { withTempDir { dir => val path = dir.toURI.getPath @@ -407,6 +426,4 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { assert(e.contains("Only Data Sources providing FileFormat are supported")) } } - - } From 4a5ff2912b15a00e7568893be0fa0b61618146c2 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Thu, 7 Sep 2017 12:05:07 -0700 Subject: [PATCH 43/47] add multi insert --- .../apache/spark/sql/hive/InsertSuite.scala | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index e17e87f3c6ed..3c2a8ae42de1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -624,6 +624,32 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } + test("multi insert overwrite to dir") { + withTempView("test_insert_table") { + spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") + + withTempDir { dir => + val pathUri = dir.toURI + + sql( + s""" + |FROM test_insert_table + |INSERT OVERWRITE DIRECTORY '${pathUri}' + |STORED AS orc + |SELECT id + |INSERT OVERWRITE DIRECTORY '${pathUri}' + |STORED AS orc + |SELECT * + """.stripMargin) + + // use orc data source to check the data of path is right. + checkAnswer( + spark.read.orc(dir.getCanonicalPath), + sql("select * from test_insert_table")) + } + } + } + test("insert overwrite to dir to illegal path") { withTempView("test_insert_table") { spark.range(10).selectExpr("id", "id AS str").createOrReplaceTempView("test_insert_table") From 449249e527e6b748e6bb111747bacab17390516c Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 8 Sep 2017 19:37:36 -0700 Subject: [PATCH 44/47] address gatorsmile's comments --- .../plans/logical/basicLogicalOperators.scala | 2 +- .../execution/command/DDLParserSuite.scala | 4 +-- .../apache/spark/sql/hive/InsertSuite.scala | 36 +++++++++++-------- 3 files changed, 25 insertions(+), 17 deletions(-) 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 3f3c4ba49a67..5eca7aa48411 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 @@ -369,7 +369,7 @@ case class InsertIntoTable( * @param overwrite If true, the existing directory will be overwritten * * Note that this plan is unresolved and has to be replaced by the concrete implementations - * sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scaladuring analysis. + * during analysis. */ case class InsertIntoDir( isLocal: Boolean, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 0375e58dfdc6..fa5172ca8a3e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -551,11 +551,11 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { parser.parsePlan(v3) match { case InsertIntoDir(_, storage, provider, query, overwrite) => assert(storage.locationUri.isDefined && provider == Some("json")) - assert(storage.properties.get("path") == Some("/tmp/file")) assert(storage.properties.get("a") == Some("1")) assert(storage.properties.get("b") == Some("0.1")) - assert(storage.properties.get("c") == Some("TRUE")) + assert(storage.properties.get("c") == Some("true")) assert(!storage.properties.contains("abc")) + assert(!storage.properties.contains("path")) case other => fail(s"Expected to parse ${classOf[InsertIntoDataSourceDirCommand].getClass.getName}" + " from query," + s"got ${other.getClass.getName}: $v1") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 45c6ff42270f..aa5cae33f5cd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -645,21 +645,29 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter withTempDir { dir => val pathUri = dir.toURI - sql( - s""" - |FROM test_insert_table - |INSERT OVERWRITE DIRECTORY '${pathUri}' - |STORED AS orc - |SELECT id - |INSERT OVERWRITE DIRECTORY '${pathUri}' - |STORED AS orc - |SELECT * - """.stripMargin) + withTempDir { dir2 => + val pathUri2 = dir2.toURI - // use orc data source to check the data of path is right. - checkAnswer( - spark.read.orc(dir.getCanonicalPath), - sql("select * from test_insert_table")) + sql( + s""" + |FROM test_insert_table + |INSERT OVERWRITE DIRECTORY '${pathUri}' + |STORED AS orc + |SELECT id + |INSERT OVERWRITE DIRECTORY '${pathUri2}' + |STORED AS orc + |SELECT * + """.stripMargin) + + // use orc data source to check the data of path is right. + checkAnswer( + spark.read.orc(dir.getCanonicalPath), + sql("select id from test_insert_table")) + + checkAnswer( + spark.read.orc(dir2.getCanonicalPath), + sql("select * from test_insert_table")) + } } } } From 79190414db6a825f4489cb71a6050a4c3aa0fd46 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Fri, 8 Sep 2017 22:00:26 -0700 Subject: [PATCH 45/47] address viirya's comment --- .../spark/sql/hive/execution/InsertIntoHiveDirCommand.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 02a5508ee47e..c41ac4d80b3a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -49,7 +49,7 @@ import org.apache.spark.sql.hive.client.HiveClientImpl * @param isLocal whether the path specified in `storage` is a local directory * @param storage storage format used to describe how the query result is stored. * @param query the logical plan representing data to write to - * @param overwrite whthere overwrites existing directory + * @param overwrite whether overwrites existing directory */ case class InsertIntoHiveDirCommand( isLocal: Boolean, From aeb5d5ebd5c29f5a1081413d941afd76f8ee93b5 Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Sat, 9 Sep 2017 08:46:24 -0700 Subject: [PATCH 46/47] address viirya's comment --- .../sql/catalyst/parser/AstBuilder.scala | 21 ++++++++++--------- .../plans/logical/basicLogicalOperators.scala | 3 +-- .../InsertIntoDataSourceDirCommand.scala | 2 +- 3 files changed, 13 insertions(+), 13 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 9c5ca54ddc3a..891f61698f17 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 @@ -180,9 +180,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging /** * Parameters used for writing query to a table: - * (tableIdentifier, partitionKeys, overwrite, exists). + * (tableIdentifier, partitionKeys, exists). */ - type InsertTableParams = (TableIdentifier, Map[String, Option[String]], Boolean, Boolean) + type InsertTableParams = (TableIdentifier, Map[String, Option[String]], Boolean) /** * Parameters used for writing query to a directory: (isLocal, CatalogStorageFormat, provider). @@ -204,11 +204,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging query: LogicalPlan): LogicalPlan = withOrigin(ctx) { ctx match { case table: InsertIntoTableContext => - val (tableIdent, partitionKeys, overwrite, exists) = visitInsertIntoTable(table) - InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, overwrite, exists) + val (tableIdent, partitionKeys, exists) = visitInsertIntoTable(table) + InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, false, exists) case table: InsertOverwriteTableContext => - val (tableIdent, partitionKeys, overwrite, exists) = visitInsertOverwriteTable(table) - InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, overwrite, exists) + val (tableIdent, partitionKeys, exists) = visitInsertOverwriteTable(table) + InsertIntoTable(UnresolvedRelation(tableIdent), partitionKeys, query, true, exists) case dir: InsertOverwriteDirContext => val (isLocal, storage, provider) = visitInsertOverwriteDir(dir) InsertIntoDir(isLocal, storage, provider, query, overwrite = true) @@ -228,7 +228,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val tableIdent = visitTableIdentifier(ctx.tableIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) - (tableIdent, partitionKeys, false, false) + (tableIdent, partitionKeys, false) } /** @@ -236,6 +236,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitInsertOverwriteTable( ctx: InsertOverwriteTableContext): InsertTableParams = withOrigin(ctx) { + assert(ctx.OVERWRITE() != null) val tableIdent = visitTableIdentifier(ctx.tableIdentifier) val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty) @@ -245,11 +246,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx) } - (tableIdent, partitionKeys, ctx.OVERWRITE() != null, ctx.EXISTS() != null) + (tableIdent, partitionKeys, ctx.EXISTS() != null) } /** - * Write to a file, returning a [[InsertIntoDir]] logical plan. + * Write to a directory, returning a [[InsertIntoDir]] logical plan. */ override def visitInsertOverwriteDir( ctx: InsertOverwriteDirContext): InsertDirParams = withOrigin(ctx) { @@ -257,7 +258,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } /** - * Write to a file, returning a [[InsertIntoDir]] logical plan. + * Write to a directory, returning a [[InsertIntoDir]] logical plan. */ override def visitInsertOverwriteHiveDir( ctx: InsertOverwriteHiveDirContext): InsertDirParams = withOrigin(ctx) { 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 5eca7aa48411..f443cd5a69de 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 @@ -377,9 +377,8 @@ case class InsertIntoDir( provider: Option[String], child: LogicalPlan, overwrite: Boolean = true) - extends LogicalPlan { + extends UnaryNode { - override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = Seq.empty override lazy val resolved: Boolean = false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala index f0d8e6ddb65f..633de4c37af9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/InsertIntoDataSourceDirCommand.scala @@ -50,7 +50,7 @@ case class InsertIntoDataSourceDirCommand( override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { assert(children.length == 1) assert(storage.locationUri.nonEmpty, "Directory path is required") - assert(!provider.isEmpty, "Data source is required") + assert(provider.nonEmpty, "Data source is required") // Create the relation based on the input logical plan: `query`. val pathOption = storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) From f93d57a224da29ee529a2d691c0d8edea1808d0b Mon Sep 17 00:00:00 2001 From: Jane Wang Date: Sat, 9 Sep 2017 08:59:52 -0700 Subject: [PATCH 47/47] address viirya's comment --- .../spark/sql/hive/execution/InsertIntoHiveDirCommand.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index c41ac4d80b3a..2110038db36a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -117,12 +117,12 @@ case class InsertIntoHiveDirCommand( fs.listStatus(tmpPath).foreach { tmpFile => fs.rename(tmpFile.getPath, writeToPath) } - - deleteExternalTmpPath(hadoopConf) } catch { case e: Throwable => throw new SparkException( "Failed inserting overwrite directory " + storage.locationUri.get, e) + } finally { + deleteExternalTmpPath(hadoopConf) } Seq.empty[Row]