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



##########
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
+
+  /**
+   * Single writer mode always has at most one writer.
+   * The output is expected to be sorted on partition and/or bucket columns 
before writing.
+   */
+  case object SingleWriter extends WriterMode
+
+  /**
+   * Concurrent writer mode before sort happens, and can have multiple 
concurrent writers
+   * for each partition and/or bucket columns.
+   */
+  case object ConcurrentWriterBeforeSort extends WriterMode
+
+  /**
+   * Concurrent writer mode after sort happens.
+   */
+  case object ConcurrentWriterAfterSort extends WriterMode
+
+  /** Wrapper class to index a unique output writer. */
+  private case class WriterIndex(
+      var partitionValues: Option[UnsafeRow],
+      var bucketId: Option[Int])
+
+  /** Wrapper class for status of a unique concurrent output writer. */
+  private case class ConcurrentWriterStatus(
+      var outputWriter: OutputWriter,
+      var recordsInFile: Long,
+      var fileCounter: Int,
+      var filePath: String)

Review comment:
       Yes, because we may create a new file if exceeding limit of number of 
records.




-- 
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:
us...@infra.apache.org



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

Reply via email to