Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 6 additions & 2 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -366,17 +366,21 @@ private[spark] object SQLConf {
"storing additional schema information in Hive's metastore.",
isPublic = false)

// Whether to perform partition discovery when loading external data sources. Default to true.
val PARTITION_DISCOVERY_ENABLED = booleanConf("spark.sql.sources.partitionDiscovery.enabled",
defaultValue = Some(true),
doc = "When true, automtically discover data partitions.")

// Whether to perform partition column type inference. Default to true.
val PARTITION_COLUMN_TYPE_INFERENCE =
booleanConf("spark.sql.sources.partitionColumnTypeInference.enabled",
defaultValue = Some(true),
doc = "When true, automatically infer the data types for partitioned columns.")

val PARTITION_MAX_FILES =
intConf("spark.sql.sources.maxConcurrentWrites",
defaultValue = Some(5),
doc = "The maximum number of concurent files to open before falling back on sorting when " +
"writing out files using dynamic partitioning.")

// The output committer class used by HadoopFsRelation. The specified class needs to be a
// subclass of org.apache.hadoop.mapreduce.OutputCommitter.
//
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* 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.datasources

import java.io.IOException
import java.util.{Date, UUID}

import scala.collection.JavaConversions.asScalaIterator

import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce._
import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter => MapReduceFileOutputCommitter, FileOutputFormat}
import org.apache.spark._
import org.apache.spark.mapred.SparkHadoopMapRedUtil
import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateProjection
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.execution.{RunnableCommand, SQLExecution}
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.StringType
import org.apache.spark.util.{Utils, SerializableConfiguration}


/**
* Inserts the results of `query` in to a relation that extends [[InsertableRelation]].
*/
private[sql] case class InsertIntoDataSource(
logicalRelation: LogicalRelation,
query: LogicalPlan,
overwrite: Boolean)
extends RunnableCommand {

override def run(sqlContext: SQLContext): Seq[Row] = {
val relation = logicalRelation.relation.asInstanceOf[InsertableRelation]
val data = DataFrame(sqlContext, query)
// Apply the schema of the existing table to the new data.
val df = sqlContext.internalCreateDataFrame(data.queryExecution.toRdd, logicalRelation.schema)
relation.insert(df, overwrite)

// Invalidate the cache.
sqlContext.cacheManager.invalidateCache(logicalRelation)

Seq.empty[Row]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like the InsertIntoDataSource is inconsistent with InsertIntoHadoopFsRelation.

  • In InsertIntoHadoopFsRelation, we do the insertion inside SQLExecution.withNewExecutionId, but here we don't.
  • Here we invalidate the cache after insertion, but InsertIntoHadoopFsRelation don't.

Is this by intentional?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The cache invalidation part is a known bug.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@chenghao-intel is trying to fix this in #8023

}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,165 @@
/*
* 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.datasources

import java.io.IOException

import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce._
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
import org.apache.spark._
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.{RunnableCommand, SQLExecution}
import org.apache.spark.sql.sources._
import org.apache.spark.util.Utils


/**
* A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending.
* Writing to dynamic partitions is also supported. Each [[InsertIntoHadoopFsRelation]] issues a
* single write job, and owns a UUID that identifies this job. Each concrete implementation of
* [[HadoopFsRelation]] should use this UUID together with task id to generate unique file path for
* each task output file. This UUID is passed to executor side via a property named
* `spark.sql.sources.writeJobUUID`.
*
* Different writer containers, [[DefaultWriterContainer]] and [[DynamicPartitionWriterContainer]]
* are used to write to normal tables and tables with dynamic partitions.
*
* Basic work flow of this command is:
*
* 1. Driver side setup, including output committer initialization and data source specific
* preparation work for the write job to be issued.
* 2. Issues a write job consists of one or more executor side tasks, each of which writes all
* rows within an RDD partition.
* 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any
* exception is thrown during task commitment, also aborts that task.
* 4. If all tasks are committed, commit the job, otherwise aborts the job; If any exception is
* thrown during job commitment, also aborts the job.
*/
private[sql] case class InsertIntoHadoopFsRelation(
@transient relation: HadoopFsRelation,
@transient query: LogicalPlan,
mode: SaveMode)
extends RunnableCommand {

override def run(sqlContext: SQLContext): Seq[Row] = {
require(
relation.paths.length == 1,
s"Cannot write to multiple destinations: ${relation.paths.mkString(",")}")

val hadoopConf = sqlContext.sparkContext.hadoopConfiguration
val outputPath = new Path(relation.paths.head)
val fs = outputPath.getFileSystem(hadoopConf)
val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory)

val pathExists = fs.exists(qualifiedOutputPath)
val doInsertion = (mode, pathExists) match {
case (SaveMode.ErrorIfExists, true) =>
throw new AnalysisException(s"path $qualifiedOutputPath already exists.")
case (SaveMode.Overwrite, true) =>
Utils.tryOrIOException {
if (!fs.delete(qualifiedOutputPath, true /* recursively */)) {
throw new IOException(s"Unable to clear output " +
s"directory $qualifiedOutputPath prior to writing to it")
}
}
true
case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) =>
true
case (SaveMode.Ignore, exists) =>
!exists
case (s, exists) =>
throw new IllegalStateException(s"unsupported save mode $s ($exists)")
}
// If we are appending data to an existing dir.
val isAppend = pathExists && (mode == SaveMode.Append)

if (doInsertion) {
val job = new Job(hadoopConf)
job.setOutputKeyClass(classOf[Void])
job.setOutputValueClass(classOf[InternalRow])
FileOutputFormat.setOutputPath(job, qualifiedOutputPath)

// A partitioned relation schema's can be different from the input logicalPlan, since
// partition columns are all moved after data column. We Project to adjust the ordering.
// TODO: this belongs in the analyzer.
val project = Project(
relation.schema.map(field => UnresolvedAttribute.quoted(field.name)), query)
val queryExecution = DataFrame(sqlContext, project).queryExecution

SQLExecution.withNewExecutionId(sqlContext, queryExecution) {
val df = sqlContext.internalCreateDataFrame(queryExecution.toRdd, relation.schema)
val partitionColumns = relation.partitionColumns.fieldNames

// Some pre-flight checks.
require(
df.schema == relation.schema,
s"""DataFrame must have the same schema as the relation to which is inserted.
|DataFrame schema: ${df.schema}
|Relation schema: ${relation.schema}
""".stripMargin)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Indentation is off.

val partitionColumnsInSpec = relation.partitionColumns.fieldNames
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems partitionColumnsInSpec and partitionColumns point to the same thing?
I think you mean the partition in InsertIntoTable? If so, it's already checked by PreWriteCheck.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, thats possible. I just copied this from the earlier code and moved it to a better place. If you want to add an analysis test to make sure this error works and then can still remove this code that would be great!

require(
partitionColumnsInSpec.sameElements(partitionColumns),
s"""Partition columns mismatch.
|Expected: ${partitionColumnsInSpec.mkString(", ")}
|Actual: ${partitionColumns.mkString(", ")}
""".stripMargin)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Indentation off.


val writerContainer = if (partitionColumns.isEmpty) {
new DefaultWriterContainer(relation, job, isAppend)
} else {
val output = df.queryExecution.executedPlan.output
val (partitionOutput, dataOutput) =
output.partition(a => partitionColumns.contains(a.name))

new DynamicPartitionWriterContainer(
relation,
job,
partitionOutput,
dataOutput,
output,
PartitioningUtils.DEFAULT_PARTITION_NAME,
sqlContext.conf.getConf(SQLConf.PARTITION_MAX_FILES),
isAppend)
}

// This call shouldn't be put into the `try` block below because it only initializes and
// prepares the job, any exception thrown from here shouldn't cause abortJob() to be called.
writerContainer.driverSideSetup()

try {
sqlContext.sparkContext.runJob(df.queryExecution.toRdd, writerContainer.writeRows _)
writerContainer.commitJob()
relation.refresh()
} catch { case cause: Throwable =>
logError("Aborting job.", cause)
writerContainer.abortJob()
throw new SparkException("Job aborted.", cause)
}
}
} else {
logInfo("Skipping insertion into a relation that already exists.")
}

Seq.empty[Row]
}
}
Loading