You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/04/20 18:30:26 UTC

[GitHub] [spark] c21 commented on a change in pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r616939922



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -247,34 +290,168 @@ class DynamicPartitionDataWriter(
     val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
     val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
 
-    if (currentPartitionValues != nextPartitionValues || currentBucketId != nextBucketId) {
+    if (currentWriterId.partitionValues != nextPartitionValues ||
+      currentWriterId.bucketId != nextBucketId) {
       // See a new partition or bucket - write to a new partition dir (or a new bucket file).
-      if (isPartitioned && currentPartitionValues != nextPartitionValues) {
-        currentPartitionValues = Some(nextPartitionValues.get.copy())
-        statsTrackers.foreach(_.newPartition(currentPartitionValues.get))
-      }
+      updateCurrentWriterStatus()
+
       if (isBucketed) {
-        currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (mode == SingleWriter || !concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
       }
 
-      fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else if (description.maxRecordsPerFile > 0 &&
+      getOrNewOutputWriter()
+    }
+
+    if (description.maxRecordsPerFile > 0 &&
       recordsInFile >= description.maxRecordsPerFile) {
       // Exceeded the threshold in terms of the number of records per file.
       // Create a new file by increasing the file counter.
       fileCounter += 1
       assert(fileCounter < MAX_FILE_COUNTER,
         s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER")
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentWriterId.partitionValues, currentWriterId.bucketId, true)
     }
     val outputRow = getOutputRow(record)
     currentWriter.write(outputRow)
     statsTrackers.foreach(_.newRow(outputRow))
     recordsInFile += 1
   }
+
+  /**
+   * Dedicated write code path when enabling concurrent writers.
+   *
+   * The process has the following step:
+   *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns.
+   *            Keep all writers open and write rows one by one.
+   *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows. Write rows
+   *            one by one, and eagerly close the writer when finishing each partition and/or
+   *            bucket.
+   */
+  def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && mode == ConcurrentWriterBeforeSort) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      resetWriterStatus()
+      val sorter = concurrentOutputWriterSpec.get.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  sealed abstract class WriterMode

Review comment:
       @cloud-fan - sounds good I agree with it. Will re-structure the code.
   
   Btw what do you think of change in `WriteTaskStatsTracker` and `BasicWriteTaskStatsTracker`? Do you have any concern with those interface change?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org