Skip to content

Commit

Permalink
moving away from withNumPartitions
Browse files Browse the repository at this point in the history
  • Loading branch information
edrevo committed Nov 14, 2019
1 parent c6b85f9 commit f3dba5e
Show file tree
Hide file tree
Showing 14 changed files with 36 additions and 56 deletions.
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 @@ -55,16 +55,6 @@ default WriteBuilder withInputDataSchema(StructType schema) {
return this;
}

/**
* Passes the number of partitions of the input data from Spark to data source.
*
* @return a new builder with the `schema`. By default it returns `this`, which means the given
* `numPartitions` is ignored. Please override this method to take the `numPartitions`.
*/
default WriteBuilder withNumPartitions(int numPartitions) {
return this;
}

/**
* Returns a {@link BatchWrite} to write data to batch source. By default this method throws
* exception, data sources must overwrite this method to provide an implementation, if the
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,7 +98,6 @@ class InMemoryTable(

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

Expand All @@ -120,12 +119,6 @@ class InMemoryTable(
this
}

override def withNumPartitions(numPartitions: Int): WriteBuilder = {
assert(!numPartitionsProvided, "numPartitions provided twice")
numPartitionsProvided = true
this
}

override def withQueryId(queryId: String): WriteBuilder = {
assert(!queryIdProvided, "queryId provided twice")
queryIdProvided = true
Expand All @@ -145,16 +138,13 @@ class InMemoryTable(
assert(
queryIdProvided,
"Query id wasn't provided before calling buildForBatch")
assert(
numPartitionsProvided,
"Number of partitions schema 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 @@ -86,7 +86,6 @@ case class CreateTableAsSelectExec(
case table: SupportsWrite =>
val writeBuilder = table.newWriteBuilder(writeOptions)
.withInputDataSchema(schema)
.withNumPartitions(rdd.getNumPartitions)
.withQueryId(UUID.randomUUID().toString)

writeBuilder match {
Expand Down Expand Up @@ -182,7 +181,6 @@ case class ReplaceTableAsSelectExec(
case table: SupportsWrite =>
val writeBuilder = table.newWriteBuilder(writeOptions)
.withInputDataSchema(schema)
.withNumPartitions(rdd.getNumPartitions)
.withQueryId(UUID.randomUUID().toString)

writeBuilder match {
Expand Down Expand Up @@ -334,13 +332,11 @@ case class WriteToDataSourceV2Exec(
trait BatchWriteHelper {
def table: SupportsWrite
def query: SparkPlan
def rdd: RDD[InternalRow]
def writeOptions: CaseInsensitiveStringMap

def newWriteBuilder(): WriteBuilder = {
table.newWriteBuilder(writeOptions)
.withInputDataSchema(query.schema)
.withNumPartitions(rdd.getNumPartitions)
.withQueryId(UUID.randomUUID().toString)
}
}
Expand All @@ -351,38 +347,36 @@ trait BatchWriteHelper {
trait V2TableWriteExec extends UnaryExecNode {
def query: SparkPlan

lazy 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
}
}

var commitProgress: Option[StreamWriterCommitProgress] = None

override def child: SparkPlan = query
override def output: Seq[Attribute] = Nil

protected def writeWithV2(batchWrite: BatchWrite): RDD[InternalRow] = {
val useCommitCoordinator = batchWrite.useCommitCoordinator
val messages = new Array[WriterCommitMessage](rdd.partitions.length)
val totalNumRowsAccumulator = new LongAccumulator()

val writerFactory = batchWrite.createBatchWriterFactory()
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 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.")

try {
sparkContext.runJob(
rdd,
rddWithNonEmptyPartitions,
(context: TaskContext, iter: Iterator[InternalRow]) =>
DataWritingSparkTask.run(writerFactory, context, iter, useCommitCoordinator),
rdd.partitions.indices,
rddWithNonEmptyPartitions.partitions.indices,
(index, result: DataWritingSparkTaskResult) => {
val commitMessage = result.writerCommitMessage
messages(index) = commitMessage
Expand Down Expand Up @@ -488,7 +482,6 @@ private[v2] trait AtomicTableWriteExec extends V2TableWriteExec with SupportsV1W
case table: SupportsWrite =>
val writeBuilder = table.newWriteBuilder(writeOptions)
.withInputDataSchema(query.schema)
.withNumPartitions(rdd.getNumPartitions)
.withQueryId(UUID.randomUUID().toString)

val writtenRows = writeBuilder match {
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 = {
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

0 comments on commit f3dba5e

Please sign in to comment.