Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-29248][SQL] provider number of partitions when creating v2 data writer factory #26591

Closed
wants to merge 1 commit into from
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010
import java.{util => ju}

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage}
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage}
import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery
import org.apache.spark.sql.types.StructType

Expand All @@ -40,7 +40,7 @@ private[kafka010] class KafkaBatchWrite(

validateQuery(schema.toAttributes, producerParams, topic)

override def createBatchWriterFactory(): KafkaBatchWriterFactory =
override def createBatchWriterFactory(info: PhysicalWriteInfo): KafkaBatchWriterFactory =
KafkaBatchWriterFactory(topic, producerParams, schema)

override def commit(messages: Array[WriterCommitMessage]): Unit = {}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010
import java.{util => ju}

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage}
import org.apache.spark.sql.connector.write.{DataWriter, PhysicalWriteInfo, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery
import org.apache.spark.sql.types.StructType
Expand All @@ -41,7 +41,8 @@ private[kafka010] class KafkaStreamingWrite(

validateQuery(schema.toAttributes, producerParams, topic)

override def createStreamingWriterFactory(): KafkaStreamWriterFactory =
override def createStreamingWriterFactory(
info: PhysicalWriteInfo): KafkaStreamWriterFactory =
KafkaStreamWriterFactory(topic, producerParams, schema)

override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@
* An interface that defines how to write the data to data source for batch processing.
*
* The writing procedure is:
* 1. Create a writer factory by {@link #createBatchWriterFactory()}, serialize and send it to all
* the partitions of the input data(RDD).
* 1. Create a writer factory by {@link #createBatchWriterFactory(PhysicalWriteInfo)}, serialize
* and send it to all the partitions of the input data(RDD).
* 2. For each partition, create the data writer, and write the data of the partition with this
* writer. If all the data are written successfully, call {@link DataWriter#commit()}. If
* exception happens during the writing, call {@link DataWriter#abort()}.
Expand All @@ -45,8 +45,10 @@ public interface BatchWrite {
*
* If this method fails (by throwing an exception), the action will fail and no Spark job will be
* submitted.
*
* @param info Physical information about the input data that will be written to this table.
*/
DataWriterFactory createBatchWriterFactory();
DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info);

/**
* Returns whether Spark should use the commit coordinator to ensure that at most one task for
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,9 @@
import org.apache.spark.sql.catalyst.InternalRow;

/**
* A factory of {@link DataWriter} returned by {@link BatchWrite#createBatchWriterFactory()},
* which is responsible for creating and initializing the actual data writer at executor side.
* A factory of {@link DataWriter} returned by
* {@link BatchWrite#createBatchWriterFactory(PhysicalWriteInfo)}, which is responsible for
* creating and initializing the actual data writer at executor side.
*
* Note that, the writer factory will be serialized and sent to executors, then the data writer
* will be created on executors and do the actual writing. So this interface must be
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.connector.write;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory;

/**
* This interface contains physical write information that data sources can use when
* generating a {@link DataWriterFactory} or a {@link StreamingDataWriterFactory}.
*/
@Evolving
public interface PhysicalWriteInfo {
/**
* The number of partitions of the input data that is going to be written.
*/
int numPartitions();
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,11 +23,12 @@
import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.connector.write.DataWriter;
import org.apache.spark.sql.connector.write.PhysicalWriteInfo;

/**
* A factory of {@link DataWriter} returned by
* {@link StreamingWrite#createStreamingWriterFactory()}, which is responsible for creating
* and initializing the actual data writer at executor side.
* {@link StreamingWrite#createStreamingWriterFactory(PhysicalWriteInfo)}, which is responsible for
* creating and initializing the actual data writer at executor side.
*
* Note that, the writer factory will be serialized and sent to executors, then the data writer
* will be created on executors and do the actual writing. So this interface must be
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,15 @@

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.write.DataWriter;
import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
import org.apache.spark.sql.connector.write.WriterCommitMessage;

/**
* An interface that defines how to write the data to data source in streaming queries.
*
* The writing procedure is:
* 1. Create a writer factory by {@link #createStreamingWriterFactory()}, serialize and send it to
* all the partitions of the input data(RDD).
* 1. Create a writer factory by {@link #createStreamingWriterFactory(PhysicalWriteInfo)},
* serialize and send it to all the partitions of the input data(RDD).
* 2. For each epoch in each partition, create the data writer, and write the data of the epoch in
* the partition with this writer. If all the data are written successfully, call
* {@link DataWriter#commit()}. If exception happens during the writing, call
Expand All @@ -48,8 +49,10 @@ public interface StreamingWrite {
*
* If this method fails (by throwing an exception), the action will fail and no Spark job will be
* submitted.
*
* @param info Information about the RDD that will be written to this data writer
*/
StreamingDataWriterFactory createStreamingWriterFactory();
StreamingDataWriterFactory createStreamingWriterFactory(PhysicalWriteInfo info);

/**
* Commits this writing job for the specified epoch with a list of commit messages. The commit
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,20 @@
/*
* 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.connector.write

private[sql] case class PhysicalWriteInfoImpl(numPartitions: Int) extends PhysicalWriteInfo
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ class InMemoryTable(
}

private abstract class TestBatchWrite extends BatchWrite {
override def createBatchWriterFactory(): DataWriterFactory = {
override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = {
BufferedRowsWriterFactory
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability, TableProvider}
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.sources.DataSourceRegister
import org.apache.spark.sql.types.StructType
Expand Down Expand Up @@ -58,7 +58,8 @@ private[noop] object NoopWriteBuilder extends WriteBuilder with SupportsTruncate
}

private[noop] object NoopBatchWrite extends BatchWrite {
override def createBatchWriterFactory(): DataWriterFactory = NoopWriterFactory
override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory =
NoopWriterFactory
override def commit(messages: Array[WriterCommitMessage]): Unit = {}
override def abort(messages: Array[WriterCommitMessage]): Unit = {}
}
Expand All @@ -74,8 +75,8 @@ private[noop] object NoopWriter extends DataWriter[InternalRow] {
}

private[noop] object NoopStreamingWrite extends StreamingWrite {
override def createStreamingWriterFactory(): StreamingDataWriterFactory =
NoopStreamingDataWriterFactory
override def createStreamingWriterFactory(
info: PhysicalWriteInfo): StreamingDataWriterFactory = NoopStreamingDataWriterFactory
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import org.apache.hadoop.mapreduce.Job

import org.apache.spark.internal.Logging
import org.apache.spark.internal.io.FileCommitProtocol
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, WriterCommitMessage}
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage}
import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult}
import org.apache.spark.sql.execution.datasources.FileFormatWriter.processStats

Expand All @@ -44,7 +44,7 @@ class FileBatchWrite(
committer.abortJob(job)
}

override def createBatchWriterFactory(): DataWriterFactory = {
override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = {
FileWriterFactory(description, committer)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, SupportsWrite, TableCatalog}
import org.apache.spark.sql.connector.expressions.Transform
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfoImpl, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, V1WriteBuilder, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
import org.apache.spark.sql.sources.{AlwaysTrue, Filter}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
Expand Down Expand Up @@ -353,28 +353,31 @@ trait V2TableWriteExec extends UnaryExecNode {
override def output: Seq[Attribute] = Nil

protected def writeWithV2(batchWrite: BatchWrite): RDD[InternalRow] = {
val writerFactory = batchWrite.createBatchWriterFactory()
val useCommitCoordinator = batchWrite.useCommitCoordinator
val rdd = query.execute()
// SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
// partition rdd to make sure we at least set up one write task to write the metadata.
val rddWithNonEmptyPartitions = if (rdd.partitions.length == 0) {
sparkContext.parallelize(Array.empty[InternalRow], 1)
} else {
rdd
val rdd: RDD[InternalRow] = {
val tempRdd = query.execute()
// SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
// partition rdd to make sure we at least set up one write task to write the metadata.
if (tempRdd.partitions.length == 0) {
sparkContext.parallelize(Array.empty[InternalRow], 1)
} else {
tempRdd
}
}
val messages = new Array[WriterCommitMessage](rddWithNonEmptyPartitions.partitions.length)
val writerFactory = batchWrite.createBatchWriterFactory(
PhysicalWriteInfoImpl(rdd.getNumPartitions))
val useCommitCoordinator = batchWrite.useCommitCoordinator
val messages = new Array[WriterCommitMessage](rdd.partitions.length)
val totalNumRowsAccumulator = new LongAccumulator()

logInfo(s"Start processing data source write support: $batchWrite. " +
s"The input RDD has ${messages.length} partitions.")

try {
sparkContext.runJob(
rddWithNonEmptyPartitions,
rdd,
(context: TaskContext, iter: Iterator[InternalRow]) =>
DataWritingSparkTask.run(writerFactory, context, iter, useCommitCoordinator),
rddWithNonEmptyPartitions.partitions.indices,
rdd.partitions.indices,
(index, result: DataWritingSparkTaskResult) => {
val commitMessage = result.writerCommitMessage
messages(index) = commitMessage
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
import org.apache.spark.sql.connector.write.streaming.StreamingWrite
import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
import org.apache.spark.sql.execution.streaming.StreamExecution
Expand All @@ -38,8 +39,10 @@ case class WriteToContinuousDataSourceExec(write: StreamingWrite, query: SparkPl
override def output: Seq[Attribute] = Nil

override protected def doExecute(): RDD[InternalRow] = {
val writerFactory = write.createStreamingWriterFactory()
val rdd = new ContinuousWriteRDD(query.execute(), writerFactory)
val queryRdd = query.execute()
val writerFactory = write.createStreamingWriterFactory(
PhysicalWriteInfoImpl(queryRdd.getNumPartitions))
val rdd = new ContinuousWriteRDD(queryRdd, writerFactory)

logInfo(s"Start processing data source write support: $write. " +
s"The input RDD has ${rdd.partitions.length} partitions.")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming.sources
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Dataset, SparkSession}
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
import org.apache.spark.sql.connector.write.WriterCommitMessage
import org.apache.spark.sql.connector.write.{PhysicalWriteInfo, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap
Expand All @@ -38,7 +38,8 @@ class ConsoleWrite(schema: StructType, options: CaseInsensitiveStringMap)
assert(SparkSession.getActiveSession.isDefined)
protected val spark = SparkSession.getActiveSession.get

def createStreamingWriterFactory(): StreamingDataWriterFactory = PackedRowWriterFactory
def createStreamingWriterFactory(info: PhysicalWriteInfo): StreamingDataWriterFactory =
PackedRowWriterFactory

override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {
// We have to print a "Batch" label for the epoch for compatibility with the pre-data source V2
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability}
import org.apache.spark.sql.connector.write.{DataWriter, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.{DataWriter, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.execution.python.PythonForeachWriter
import org.apache.spark.sql.types.StructType
Expand Down Expand Up @@ -72,7 +72,8 @@ case class ForeachWriterTable[T](
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}

override def createStreamingWriterFactory(): StreamingDataWriterFactory = {
override def createStreamingWriterFactory(
info: PhysicalWriteInfo): StreamingDataWriterFactory = {
val rowConverter: InternalRow => T = converter match {
case Left(enc) =>
val boundEnc = enc.resolveAndBind(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.execution.streaming.sources

import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage}
import org.apache.spark.sql.connector.write.{BatchWrite, DataWriter, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}

/**
Expand All @@ -36,8 +36,8 @@ class MicroBatchWrite(eppchId: Long, val writeSupport: StreamingWrite) extends B
writeSupport.abort(eppchId, messages)
}

override def createBatchWriterFactory(): DataWriterFactory = {
new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory())
override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = {
new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory(info))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils
import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability}
import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory, PhysicalWriteInfo, SupportsTruncate, WriteBuilder, WriterCommitMessage}
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.execution.streaming.Sink
import org.apache.spark.sql.types.StructType
Expand Down Expand Up @@ -140,7 +140,7 @@ class MemoryStreamingWrite(
val sink: MemorySink, schema: StructType, needTruncate: Boolean)
extends StreamingWrite {

override def createStreamingWriterFactory: MemoryWriterFactory = {
override def createStreamingWriterFactory(info: PhysicalWriteInfo): MemoryWriterFactory = {
MemoryWriterFactory(schema)
}

Expand Down