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] Pass in number of partitions to WriteBuilder #25945

Closed
wants to merge 4 commits 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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ private[kafka010] class KafkaBatchWrite(

validateQuery(schema.toAttributes, producerParams, topic)

override def createBatchWriterFactory(): KafkaBatchWriterFactory =
override def createBatchWriterFactory(numPartitions: Int): 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 @@ -41,7 +41,7 @@ private[kafka010] class KafkaStreamingWrite(

validateQuery(schema.toAttributes, producerParams, topic)

override def createStreamingWriterFactory(): KafkaStreamWriterFactory =
override def createStreamingWriterFactory(numPartitions: Int): 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 @@ -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 numPartitions The number of partitions of the RDD that is going to be written.
*/
DataWriterFactory createBatchWriterFactory();
DataWriterFactory createBatchWriterFactory(int numPartitions);

/**
* 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 @@ -48,8 +48,10 @@ public interface StreamingWrite {
*
* If this method fails (by throwing an exception), the action will fail and no Spark job will be
* submitted.
*
* @param numPartitions The number of partitions of the RDD that is going to be written.
*/
StreamingDataWriterFactory createStreamingWriterFactory();
StreamingDataWriterFactory createStreamingWriterFactory(int numPartitions);

/**
* 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
Expand Up @@ -98,6 +98,8 @@ class InMemoryTable(

new WriteBuilder with SupportsTruncate with SupportsOverwrite with SupportsDynamicOverwrite {
private var writer: BatchWrite = Append
private var queryIdProvided = false
private var inputDataSchemaProvided = false

override def truncate(): WriteBuilder = {
assert(writer == Append)
Expand All @@ -117,12 +119,32 @@ class InMemoryTable(
this
}

override def buildForBatch(): BatchWrite = writer
override def withQueryId(queryId: String): WriteBuilder = {
assert(!queryIdProvided, "queryId provided twice")
Copy link
Contributor

Choose a reason for hiding this comment

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

later on can we continue the work in #25990 ? It still has value to give stronger guarantees. So that implementations don't need to do check like this.

queryIdProvided = true
this
}

override def withInputDataSchema(schema: StructType): WriteBuilder = {
assert(!queryIdProvided, "schema provided twice")
inputDataSchemaProvided = true
this
}

override def buildForBatch(): BatchWrite = {
assert(
inputDataSchemaProvided,
"Input data schema wasn't provided before calling buildForBatch")
assert(
queryIdProvided,
"Query id wasn't provided before calling buildForBatch")
writer
}
}
}

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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ private[noop] object NoopWriteBuilder extends WriteBuilder with SupportsTruncate
}

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

private[noop] object NoopStreamingWrite extends StreamingWrite {
override def createStreamingWriterFactory(): StreamingDataWriterFactory =
override def createStreamingWriterFactory(numPartitions: Int): 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 @@ -44,7 +44,7 @@ class FileBatchWrite(
committer.abortJob(job)
}

override def createBatchWriterFactory(): DataWriterFactory = {
override def createBatchWriterFactory(numPartitions: Int): DataWriterFactory = {
FileWriterFactory(description, committer)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -353,8 +353,8 @@ 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.
Expand All @@ -365,6 +365,8 @@ trait V2TableWriteExec extends UnaryExecNode {
}
val messages = new Array[WriterCommitMessage](rddWithNonEmptyPartitions.partitions.length)
val totalNumRowsAccumulator = new LongAccumulator()
val writerFactory = batchWrite.createBatchWriterFactory(
rddWithNonEmptyPartitions.partitions.length)

logInfo(s"Start processing data source write support: $batchWrite. " +
s"The input RDD has ${messages.length} partitions.")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,8 +38,9 @@ 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(queryRdd.partitions.length)
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 @@ -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(numPartitions: Int): 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 @@ -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(
numPartitions: Int): 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 @@ -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(numPartitions: Int): DataWriterFactory = {
Copy link
Contributor

Choose a reason for hiding this comment

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

sorry to come up with this at the last minute: can we create a PhysicalWriteInfo interface? In case we want to add more physical information in the future.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

No problem! Since we still want to move forward with the interface-based approach, I've decided to evolve #25990 to include both the PhysicalWriteInfo as well as WriteInfo

new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory(numPartitions))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ class MemoryStreamingWrite(
val sink: MemorySink, schema: StructType, needTruncate: Boolean)
extends StreamingWrite {

override def createStreamingWriterFactory: MemoryWriterFactory = {
override def createStreamingWriterFactory(numPartitions: Int): MemoryWriterFactory = {
MemoryWriterFactory(schema)
}

Expand Down