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 14:13:35 UTC

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

cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r616724095



##########
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.

Review comment:
       `columns` -> `values`?




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