-
Notifications
You must be signed in to change notification settings - Fork 28.3k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
1 parent
d53e11f
commit 5358ad4
Showing
15 changed files
with
459 additions
and
46 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
46 changes: 46 additions & 0 deletions
46
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
/* | ||
* 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.v2 | ||
|
||
import org.apache.hadoop.mapreduce.Job | ||
|
||
import org.apache.spark.internal.io.FileCommitProtocol | ||
import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult} | ||
import org.apache.spark.sql.sources.v2.writer._ | ||
import org.apache.spark.util.SerializableConfiguration | ||
|
||
class FileBatchWrite( | ||
job: Job, | ||
description: WriteJobDescription, | ||
committer: FileCommitProtocol) | ||
extends BatchWrite { | ||
override def commit(messages: Array[WriterCommitMessage]): Unit = { | ||
committer.commitJob(job, messages.map(_.asInstanceOf[WriteTaskResult].commitMsg)) | ||
} | ||
|
||
override def useCommitCoordinator(): Boolean = false | ||
|
||
override def abort(messages: Array[WriterCommitMessage]): Unit = { | ||
committer.abortJob(job) | ||
} | ||
|
||
override def createBatchWriterFactory(): DataWriterFactory = { | ||
val conf = new SerializableConfiguration(job.getConfiguration) | ||
FileWriterFactory(description, committer, conf) | ||
} | ||
} | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
140 changes: 140 additions & 0 deletions
140
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,140 @@ | ||
/* | ||
* 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.v2 | ||
|
||
import java.util.{Optional, UUID} | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.hadoop.fs.Path | ||
import org.apache.hadoop.mapreduce.Job | ||
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat | ||
|
||
import org.apache.spark.internal.io.{FileCommitProtocol, HadoopMapReduceCommitProtocol} | ||
import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} | ||
import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, DataSource, OutputWriterFactory, WriteJobDescription} | ||
import org.apache.spark.sql.execution.metric.SQLMetric | ||
import org.apache.spark.sql.internal.SQLConf | ||
import org.apache.spark.sql.sources.v2.{DataSourceOptions, SupportsBatchWrite} | ||
import org.apache.spark.sql.sources.v2.writer.{BatchWrite, SupportsSaveMode, WriteBuilder} | ||
import org.apache.spark.sql.types.{DataType, StructType} | ||
import org.apache.spark.util.SerializableConfiguration | ||
|
||
abstract class FileWriteBuilder(options: DataSourceOptions) | ||
extends WriteBuilder with SupportsSaveMode { | ||
private var schema: StructType = _ | ||
private var queryId: String = _ | ||
private var mode: SaveMode = _ | ||
|
||
override def withInputDataSchema(schema: StructType): WriteBuilder = { | ||
this.schema = schema | ||
this | ||
} | ||
|
||
override def withQueryId(queryId: String): WriteBuilder = { | ||
this.queryId = queryId | ||
this | ||
} | ||
|
||
override def mode(mode: SaveMode): WriteBuilder = { | ||
this.mode = mode | ||
this | ||
} | ||
|
||
override def buildForBatch(): BatchWrite = { | ||
assert(schema != null, "Missing input data schema") | ||
assert(queryId != null, "Missing query ID") | ||
assert(mode != null, "Missing save mode") | ||
assert(options.paths().length == 1) | ||
DataSource.validateSchema(schema) | ||
val pathName = options.paths().head | ||
val path = new Path(pathName) | ||
val sparkSession = SparkSession.active | ||
val optionsAsScala = options.asMap().asScala.toMap | ||
val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(optionsAsScala) | ||
val job = Job.getInstance(hadoopConf) | ||
val committer = FileCommitProtocol.instantiate( | ||
sparkSession.sessionState.conf.fileCommitProtocolClass, | ||
jobId = java.util.UUID.randomUUID().toString, | ||
outputPath = pathName) | ||
|
||
job.setOutputKeyClass(classOf[Void]) | ||
job.setOutputValueClass(classOf[InternalRow]) | ||
FileOutputFormat.setOutputPath(job, path) | ||
|
||
val caseInsensitiveOptions = CaseInsensitiveMap(optionsAsScala) | ||
// Note: prepareWrite has side effect. It sets "job". | ||
val outputWriterFactory = | ||
prepareWrite(sparkSession.sessionState.conf, job, caseInsensitiveOptions, schema) | ||
val allColumns = schema.toAttributes | ||
lazy val metrics: Map[String, SQLMetric] = BasicWriteJobStatsTracker.metrics | ||
val serializableHadoopConf = new SerializableConfiguration(hadoopConf) | ||
val statsTracker = new BasicWriteJobStatsTracker(serializableHadoopConf, metrics) | ||
// TODO: after partitioning is supported in V2: | ||
// 1. filter out partition columns in `dataColumns`. | ||
// 2. Don't use Seq.empty for `partitionColumns`. | ||
val description = new WriteJobDescription( | ||
uuid = UUID.randomUUID().toString, | ||
serializableHadoopConf = new SerializableConfiguration(job.getConfiguration), | ||
outputWriterFactory = outputWriterFactory, | ||
allColumns = allColumns, | ||
dataColumns = allColumns, | ||
partitionColumns = Seq.empty, | ||
bucketIdExpression = None, | ||
path = pathName, | ||
customPartitionLocations = Map.empty, | ||
maxRecordsPerFile = caseInsensitiveOptions.get("maxRecordsPerFile").map(_.toLong) | ||
.getOrElse(sparkSession.sessionState.conf.maxRecordsPerFile), | ||
timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) | ||
.getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone), | ||
statsTrackers = Seq(statsTracker) | ||
) | ||
|
||
val fs = path.getFileSystem(hadoopConf) | ||
mode match { | ||
case SaveMode.ErrorIfExists if (fs.exists(path)) => | ||
val qualifiedOutputPath = path.makeQualified(fs.getUri, fs.getWorkingDirectory) | ||
throw new AnalysisException(s"path $qualifiedOutputPath already exists.") | ||
|
||
case SaveMode.Ignore if (fs.exists(path)) => | ||
null | ||
|
||
case SaveMode.Overwrite => | ||
committer.deleteWithJob(fs, path, true) | ||
committer.setupJob(job) | ||
new FileBatchWrite(job, description, committer) | ||
|
||
case _ => | ||
committer.setupJob(job) | ||
new FileBatchWrite(job, description, committer) | ||
} | ||
} | ||
|
||
/** | ||
* Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can | ||
* be put here. For example, user defined output committer can be configured here | ||
* by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. | ||
*/ | ||
def prepareWrite( | ||
sqlConf: SQLConf, | ||
job: Job, | ||
options: Map[String, String], | ||
dataSchema: StructType): OutputWriterFactory | ||
} | ||
|
58 changes: 58 additions & 0 deletions
58
...core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,58 @@ | ||
/* | ||
* 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.v2 | ||
|
||
import java.util.Date | ||
|
||
import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskID, TaskType} | ||
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl | ||
|
||
import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.execution.datasources.{DynamicPartitionDataWriter, EmptyDirectoryDataWriter, SingleDirectoryDataWriter, WriteJobDescription} | ||
import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} | ||
import org.apache.spark.util.SerializableConfiguration | ||
|
||
case class FileWriterFactory ( | ||
description: WriteJobDescription, | ||
committer: FileCommitProtocol, | ||
conf: SerializableConfiguration) extends DataWriterFactory { | ||
override def createWriter(partitionId: Int, realTaskId: Long): DataWriter[InternalRow] = { | ||
val taskAttemptContext = createTaskAttemptContext(partitionId) | ||
committer.setupTask(taskAttemptContext) | ||
if (description.partitionColumns.isEmpty) { | ||
new SingleDirectoryDataWriter(description, taskAttemptContext, committer) | ||
} else { | ||
new DynamicPartitionDataWriter(description, taskAttemptContext, committer) | ||
} | ||
} | ||
|
||
private def createTaskAttemptContext(partitionId: Int): TaskAttemptContextImpl = { | ||
val jobId = SparkHadoopWriterUtils.createJobID(new Date, 0) | ||
val taskId = new TaskID(jobId, TaskType.MAP, partitionId) | ||
val taskAttemptId = new TaskAttemptID(taskId, 0) | ||
// Set up the configuration object | ||
val hadoopConf = conf.value | ||
hadoopConf.set("mapreduce.job.id", jobId.toString) | ||
hadoopConf.set("mapreduce.task.id", taskId.toString) | ||
hadoopConf.set("mapreduce.task.attempt.id", taskAttemptId.toString) | ||
hadoopConf.setBoolean("mapreduce.task.ismap", true) | ||
hadoopConf.setInt("mapreduce.task.partition", 0) | ||
|
||
new TaskAttemptContextImpl(hadoopConf, taskAttemptId) | ||
} | ||
} |
Oops, something went wrong.