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/16 01:06:40 UTC

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

c21 opened a new pull request #32198:
URL: https://github.com/apache/spark/pull/32198


   
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   This is a re-proposal of https://github.com/apache/spark/pull/23163. Currently spark always requires a [local sort](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala#L188) before writing to output table with dynamic partition/bucket columns. The sort can be unnecessary if cardinality of partition/bucket values is small, and can be avoided by keeping multiple output writers concurrently.
   
   This PR introduces a config `spark.sql.maxConcurrentOutputWriters` (which disables this feature by default), where user can tune the maximal number of concurrent writers. The config is needed here as we cannot keep arbitrary number of writers in task memory which can cause OOM (especially for Parquet/ORC vectorization writer).
   
   The feature is to first use concurrent writers to write rows. If the number of writers exceeds the above config specified limit. Sort rest of rows and write rows one by one (See `DynamicPartitionDataWriter.writeWithIterator()`).
   
   In addition, interface `WriteTaskStatsTracker` and its implementation `BasicWriteTaskStatsTracker` are also changed because previously they are relying on the assumption that only one writer is active for writing dynamic partitions and bucketed table.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   Avoid the sort before writing output for dynamic partitioned query and bucketed table.
   Help improve CPU and IO performance for these queries.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   No.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   Added unit test in `DataFrameReaderWriterSuite.scala`.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826581208


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42457/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618213229



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)

Review comment:
       makes sense, let's leave it then




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r616740784



##########
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:
       does it mean the latest file path?




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825279818






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827124409


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42490/
   


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827353621


   Thank you @cloud-fan for all the dedicated help and careful review!
   Thank you @imback82 and @ulysses-you for commenting and review too!


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827326207


   thanks, merging to master!


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823822517






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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618784628



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size > concurrentOutputWriterSpec.maxWriters && !sorted) {

Review comment:
       Well I agree this looks a bit weird, but I think it does not matter too much. The documentation can be understood like `the maximum number of output file writers to use concurrently, i.e. to use during concurrent write phase`. If we set the config to `2`, at most `2` writers will be used during concurrent write phase, i.e. concurrently. When the 3rd writer is created, we fallback to sort-based write phase.
   
   If we make it to `oncurrentWriters.size >= concurrentOutputWriterSpec.maxWriters`, it's kind of surprising to users when the config is set to `2`, only 1 writer is used during concurrent write phase. 




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824483915


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137749/
   


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618177561



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)

Review comment:
       > Can we always create a new WriterIndex instance when needed and make WriterIndex immutable?
   
   I am worried about that we may need to create a new `WriterIndex` a lot as data layout has no order and it may switch between writers per each row a lot, and in the worst case we need to create a new `WriterIndex` per row to lookup in the map, and has GC issue.
   
   If we are worried about correctness, is test in `DataFrameReaderWriterSuite.scala` releasing your concern?




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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826542139


   **[Test build #137935 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137935/testReport)** for PR 32198 at commit [`2895837`](https://github.com/apache/spark/commit/2895837e8d80e371388a23207f3c897808e62a55).


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r620081749



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -52,19 +53,35 @@ abstract class FileFormatDataWriter(
   protected val statsTrackers: Seq[WriteTaskStatsTracker] =
     description.statsTrackers.map(_.newTaskInstance())
 
-  protected def releaseResources(): Unit = {
+  /** Release resources of `currentWriter`. */
+  protected def releaseCurrentWriter(): Unit = {
     if (currentWriter != null) {
       try {
         currentWriter.close()
+        statsTrackers.foreach(_.closeFile(currentWriter.path()))
       } finally {
         currentWriter = null
       }
     }
   }
 
-  /** Writes a record */
+  /** Release all resources. */
+  protected def releaseResources(): Unit = {
+    // Call `releaseCurrentWriter()` by default, as this is the only resource to be released.
+    releaseCurrentWriter()
+  }
+
+  /** Writes a record. */
   def write(record: InternalRow): Unit

Review comment:
       ah I see, let's keep it.




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618157474



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0

Review comment:
       shall we reset `recordsInFile` as well?




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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823754660


   @cloud-fan - updated the PR to keep single and concurrent writers implementation separately. The PR is ready for review again, thanks.


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618157080



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      newOutputWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))

Review comment:
       It's a new writer, the initial value of currentWriter and recordsInFile should be null and 0?




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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r620578802



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -243,6 +273,52 @@ class DynamicPartitionDataWriter(
     statsTrackers.foreach(_.newFile(currentPath))
   }
 
+  /**
+   * Increase the file counter and open a new OutputWriter.
+   * This is used when number of records records exceeding limit.
+   *
+   * @param partitionValues the partition which all tuples being written by this `OutputWriter`
+   *                        belong to
+   * @param bucketId the bucket which all tuples being written by this `OutputWriter` belong to
+   */
+  protected def increaseFileCounter(

Review comment:
       @cloud-fan - sure, updated.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +333,199 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer)
+  with Logging {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+
+  /**
+   * The index for current writer. Intentionally make the index mutable and reusable.
+   * Avoid JVM GC issue when many short-living `WriterIndex` objects are created
+   * if switching between concurrent writers frequently.
+   */
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      setupCurrentWriterUsingMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def setupCurrentWriterUsingMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      if (!sorted) {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters,
+          s"Number of concurrent output file writers is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}")
+      } else {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1,
+          s"Number of output file writers after sort is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}")
+      }
+      concurrentWriters.put(
+        currentWriterId.copy(),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size >= concurrentOutputWriterSpec.maxWriters && !sorted) {
+        // Fall back to sort-based sequential writer mode.
+        logInfo(s"Number of concurrent writers ${concurrentWriters.size} reaches the threshold. " +
+          "Fall back from concurrent writers to sort-based sequential writer. You may change " +
+          s"threshold with configuration ${SQLConf.MAX_CONCURRENT_OUTPUT_FILE_WRITERS.key}")
+        sorted = true
+      }
+    }
+  }
+
+  /**
+   * Clear the current writer status in map.
+   */
+  private def clearCurrentWriterStatus(): Unit = {
+    if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) {
+      updateCurrentWriterStatusInMap()

Review comment:
       I wish I could do it to tight the logic more closely, but unfortunately no. We need to write a record (`writeRecord`) between (1).set the `sorted` to true (`setupCurrentWriterUsingMap`) and (2).clean up current writer status (`clearCurrentWriterStatus`).
   
   `writeRecord` will change the status of `recordsInFile` to be increased by 1.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820847632


   **[Test build #137451 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137451/testReport)** for PR 32198 at commit [`18f2851`](https://github.com/apache/spark/commit/18f285160bdca3c7958c5446c5d6fbcbef84b934).


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825151839


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42350/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617340405



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int,
+    var latestFilePath: String)
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null

Review comment:
       is it possible that `currentWriter` is not null when we reach here?




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824634547


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42313/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827088743


   **[Test build #137970 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137970/testReport)** for PR 32198 at commit [`efe026c`](https://github.com/apache/spark/commit/efe026cc797db994787b6a556aee699d6a79d4af).


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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823801502


   **[Test build #137716 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137716/testReport)** for PR 32198 at commit [`b527e37`](https://github.com/apache/spark/commit/b527e371ba94eb5b8d25eb1dffb35ffd9331686d).


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617889174



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
##########
@@ -273,14 +289,25 @@ object FileFormatWriter extends Logging {
       } else if (description.partitionColumns.isEmpty && description.bucketIdExpression.isEmpty) {
         new SingleDirectoryDataWriter(description, taskAttemptContext, committer)
       } else {
-        new DynamicPartitionDataWriter(description, taskAttemptContext, committer)
+        concurrentOutputWriterSpec match {
+          case Some(spec) =>
+            new DynamicPartitionDataConcurrentWriter(
+              description, taskAttemptContext, committer, spec)
+          case _ =>
+            new DynamicPartitionDataSingleWriter(description, taskAttemptContext, committer)
+        }
       }
 
     try {
       Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
         // Execute the task to write rows out and commit the task.
-        while (iterator.hasNext) {
-          dataWriter.write(iterator.next())
+        dataWriter match {
+          case w: DynamicPartitionDataConcurrentWriter =>
+            w.writeWithIterator(iterator)

Review comment:
       @cloud-fan - ~wondering what's the benefit of doing it? Updated anyway now~. After a second thought I think I get it. You want to avoid the pattern matching here, updated.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820864605






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


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

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618794911



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(

Review comment:
       The only concern I had was there is no "clear" guard against the `put`; e.g., the check for size in the next line doesn't guard against this "put" - it sets `sorted` to `true` and based on the flag, it removes current writer from `concurrentWriters`, etc. So it was bit hard for me to follow. :)




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825116985


   **[Test build #137821 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137821/testReport)** for PR 32198 at commit [`1e8d447`](https://github.com/apache/spark/commit/1e8d447e8c8d6a4cd9954a719d8a55907bcb3e5d).


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826757436


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137935/
   


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824531319


   The unit test failure is `org.apache.spark.sql.catalyst.optimizer.ReplaceNullWithFalseInPredicateSuite`, which is not related to this PR. https://github.com/apache/spark/pull/32157 introduced the failure and https://github.com/apache/spark/pull/32278 fixed it.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820951541


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137451/
   


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-828544330


   > One more thing, how much does this improve the write? Local sorts before the write are typically not too bad if you look at the cycles spend during the write. A much bigger target here would be to properly interleave I/O and CPU operations. You sort of achieve that by having multiple writers, but it IMO feels like quite a big hammer.
   
   I will add a benchmark for this as a followup.
   
   IMHO how much this can improve thing is really depending on query shape (cardinality of dynamic partitions and buckets). In one environment, if most queries having low number of partitions and users set buckets relatively small, this feature can help more. If in another environment, query tends to write a lot of partitions and users set buckets quite large, this feature helps less. We do see benefit for improving query internally and people raised the request in spark dev as well.


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r616730665



##########
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:
       This abstraction is a bit confusing. single writer or concurrent writers are like a mode that is decided statically. before-sort and after-sort are more like runtime states instead of mode.
   
   I'd expect different `FileFormatDataWriter` implementations for single and concurrent writers, and the concurrent writers implementation has a boolean state to indicate before and after sort.




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825279818






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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823874309


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137714/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617339198



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 WriterStatus(

Review comment:
       its fields are all `var`, we can make it a class instead of case class.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826577572


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42457/
   


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617935831



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int,
+    var latestFilePath: String)
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null

Review comment:
       Actually I found a bug that I don't update writer status in map, if creating a new writer when exceeding number of records limit per file. Restructured the code to fix it, and added the test for the case.




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617338863



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent output writer. */
+  private case class WriterIndex(

Review comment:
       seems `WriterIndex` is immutable, we don't need `var`




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827113032






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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618179726



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      newOutputWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))

Review comment:
       `currentWriter` is initialized in `newOutputWriter` above.




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617348669



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int,
+    var latestFilePath: String)
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    if (currentWriterId.partitionValues != nextPartitionValues ||
+      currentWriterId.bucketId != nextBucketId) {
+      // See a new partition or bucket - write to a new partition dir (or a new bucket file).
+      updateCurrentWriterStatusInMap()
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    checkRecordsInFile(currentWriterId.partitionValues, currentWriterId.bucketId)
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status when a new writer is needed for writing row.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) {
+      if (!sortBased) {
+        // Update writer status in concurrent writers map, because the writer is probably needed
+        // again later for writing other rows.

Review comment:
       Is it a performance concern that we only update the writer status when it's going to be put back into the map? What if we update the status per row after calling `writeRecord`?




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618215664



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0

Review comment:
       Let's make the name clearer. How about `renewCurrentWriter` or something similar?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826757436


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137935/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825371987


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137831/
   


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


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

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824634547


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42313/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-821009649


   **[Test build #137459 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137459/testReport)** for PR 32198 at commit [`4b2801b`](https://github.com/apache/spark/commit/4b2801b9ccf5789c928d0105b04634e26924cf0d).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  case class ConcurrentOutputWriterSpec(`


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


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

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618770419



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size > concurrentOutputWriterSpec.maxWriters && !sorted) {

Review comment:
       nit: Looks like we will be off by 1? (`.doc("Maximum number of output file writers to use concurrently.`)

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None

Review comment:
       Do we need to go thru these even after `sorted` is set to `true`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(

Review comment:
       Maybe add an assert before this (`assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters)`), since it depends on `sorted` and `concurrentWriters.remove` outside this function?




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r616730665



##########
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:
       This abstraction is a bit confusing. single writer or concurrent writers are like a mode that is decided statically. before-sort and after-sort are more like runtime states instead of mode.
   
   I'd expect different `FileFormatDataWriter` implementations for single and concurrent writers, and the concurrent writers has a boolean state to indicate before and after sort.




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820886083


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42033/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823981409


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137716/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827219471


   **[Test build #137970 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137970/testReport)** for PR 32198 at commit [`efe026c`](https://github.com/apache/spark/commit/efe026cc797db994787b6a556aee699d6a79d4af).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-821022074


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137459/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824422127


   **[Test build #137753 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137753/testReport)** for PR 32198 at commit [`52c6160`](https://github.com/apache/spark/commit/52c6160fc017afba233478271ca69ea40e799ddb).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823760548


   **[Test build #137714 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137714/testReport)** for PR 32198 at commit [`0442f05`](https://github.com/apache/spark/commit/0442f0506ee1cd694842480da2e2817ae0be04c9).


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r619982507



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)

Review comment:
       @cloud-fan - sure, added.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      if (!sorted) {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters,
+          s"Number of concurrent output file writers is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}")
+      } else {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1,
+          s"Number of output file writers after sort is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}")
+      }
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),

Review comment:
       @cloud-fan - updated.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      if (!sorted) {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters,
+          s"Number of concurrent output file writers is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}")
+      } else {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1,
+          s"Number of output file writers after sort is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}")
+      }
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size >= concurrentOutputWriterSpec.maxWriters && !sorted) {
+        // Fall back to sort-based sequential writer mode.
+        sorted = true

Review comment:
       @ulysses-you - good callout. Updated with a info logging. I think it makes sense to account how many tasks have been fallbacked and show on UI, I will take that as a followup.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()

Review comment:
       @cloud-fan - updated.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
##########
@@ -1219,4 +1219,38 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with
       }
     }
   }
+
+  test("SPARK-26164: Allow concurrent writers for multiple partitions and buckets") {
+    withTable("t1", "t2") {
+      val df = spark.range(200).map(_ => {
+        val n = scala.util.Random.nextInt

Review comment:
       @cloud-fan - good call. Updated.




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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-828537591


   > this doesn't do any sort of memory tracking right?
   
   Yes. It seems to me there's no way to track the memory usage accurately because writer is using on-heap memory. And we need memory usage information available to retrieve from each individual writer implementation (Parquet, ORC, Aveo, etc), which is not the case right now.
   
   One immature idea though is to look at executor JVM heap memory usage (which I think should already be captured).
   
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-821022074


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137459/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824425619


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42276/
   


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618786408



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None

Review comment:
       @imback82 - yes we need. Even in sort-based write, for every row we need to get partition values and/or bucket id, as we need to check if these values of current row are same as previous row's or not. If not same, a new output writer is needed. This is same behavior as original `DynamicPartitionDataWriter.write()`.




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618144501



##########
File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriter.scala
##########
@@ -84,4 +84,6 @@ private[avro] class AvroOutputWriter(
   }
 
   override def close(): Unit = recordWriter.close(context)
+
+  override def path(): String = path

Review comment:
       we can simply change `path: String` to `val path: String` in the constructor.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825275679






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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618993171



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      if (!sorted) {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters,
+          s"Number of concurrent output file writers is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}")
+      } else {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1,
+          s"Number of output file writers after sort is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}")
+      }
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),

Review comment:
       nit: `currentWriterId.copy()`




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824624175


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42313/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826581208


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42457/
   


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


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

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618795430



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size > concurrentOutputWriterSpec.maxWriters && !sorted) {

Review comment:
       Yea, it's a nit. (I think we can do a check before `put`, but it's OK).




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820884919


   **[Test build #137459 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137459/testReport)** for PR 32198 at commit [`4b2801b`](https://github.com/apache/spark/commit/4b2801b9ccf5789c928d0105b04634e26924cf0d).


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824550511


   Rebased to latest master.


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


[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

Posted by GitBox <gi...@apache.org>.
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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618956037



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -52,19 +53,35 @@ abstract class FileFormatDataWriter(
   protected val statsTrackers: Seq[WriteTaskStatsTracker] =
     description.statsTrackers.map(_.newTaskInstance())
 
-  protected def releaseResources(): Unit = {
+  /** Release resources of `currentWriter`. */
+  protected def releaseCurrentWriter(): Unit = {
     if (currentWriter != null) {
       try {
         currentWriter.close()
+        statsTrackers.foreach(_.closeFile(currentWriter.path()))
       } finally {
         currentWriter = null
       }
     }
   }
 
-  /** Writes a record */
+  /** Release all resources. */
+  protected def releaseResources(): Unit = {
+    // Call `releaseCurrentWriter()` by default, as this is the only resource to be released.
+    releaseCurrentWriter()
+  }
+
+  /** Writes a record. */
   def write(record: InternalRow): Unit

Review comment:
       This can be `protected` now.




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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823760548


   **[Test build #137714 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137714/testReport)** for PR 32198 at commit [`0442f05`](https://github.com/apache/spark/commit/0442f0506ee1cd694842480da2e2817ae0be04c9).


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823829022


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42244/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparksFyz commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-1062764141


   > > One more thing, how much does this improve the write? Local sorts before the write are typically not too bad if you look at the cycles spend during the write. A much bigger target here would be to properly interleave I/O and CPU operations. You sort of achieve that by having multiple writers, but it IMO feels like quite a big hammer.
   > 
   > I will add a benchmark for this as a followup.
   > 
   > IMHO how much this can improve thing is really depending on query shape (cardinality of dynamic partitions and buckets). In one environment, if most queries having low number of partitions and users set buckets relatively small, this feature can help more. If in another environment, query tends to write a lot of partitions and users set buckets quite large, this feature helps less. We do see benefit for improving query internally and people raised the request in spark dev as well.
   
   Hi~ Any link for benchmark? Thanks~


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824567051


   **[Test build #137784 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137784/testReport)** for PR 32198 at commit [`a4ed53b`](https://github.com/apache/spark/commit/a4ed53b702f1514de66a5696989abfc85e14a551).


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618810901



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size > concurrentOutputWriterSpec.maxWriters && !sorted) {

Review comment:
       Updated the condition and doc in `SQLConf` from `exceeds` to `reaches`, so now the implementation and documentation is more consistent.




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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820836772


   cc @cloud-fan and @maropu could you help take a look when you have time? Thanks.


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820867956


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42026/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617336198



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -56,6 +58,7 @@ abstract class FileFormatDataWriter(
     if (currentWriter != null) {

Review comment:
       I'd like to add a new method in the base class
   ```
   protected def releaseCurrentWriter() ...
   ```
   Then `releaseResources` simply calls `releaseCurrentWriter`. The child should call `releaseCurrentWriter` when needed, instead of `super[FileFormatDataWriter].releaseResources()`, which looks pretty weird.




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823778062


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42242/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823874309


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137714/
   


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


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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


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

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820872195


   @c21 would you mind rebasing w/ the latest master branch? Seems like your branch is based on the old master branch.


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823801502


   **[Test build #137716 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137716/testReport)** for PR 32198 at commit [`b527e37`](https://github.com/apache/spark/commit/b527e371ba94eb5b8d25eb1dffb35ffd9331686d).


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r620086074



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +333,199 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer)
+  with Logging {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+
+  /**
+   * The index for current writer. Intentionally make the index mutable and reusable.
+   * Avoid JVM GC issue when many short-living `WriterIndex` objects are created
+   * if switching between concurrent writers frequently.
+   */
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      setupCurrentWriterUsingMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def setupCurrentWriterUsingMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      if (!sorted) {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters,
+          s"Number of concurrent output file writers is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}")
+      } else {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1,
+          s"Number of output file writers after sort is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}")
+      }
+      concurrentWriters.put(
+        currentWriterId.copy(),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size >= concurrentOutputWriterSpec.maxWriters && !sorted) {
+        // Fall back to sort-based sequential writer mode.
+        logInfo(s"Number of concurrent writers ${concurrentWriters.size} reaches the threshold. " +
+          "Fall back from concurrent writers to sort-based sequential writer. You may change " +
+          s"threshold with configuration ${SQLConf.MAX_CONCURRENT_OUTPUT_FILE_WRITERS.key}")
+        sorted = true
+      }
+    }
+  }
+
+  /**
+   * Clear the current writer status in map.
+   */
+  private def clearCurrentWriterStatus(): Unit = {
+    if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) {
+      updateCurrentWriterStatusInMap()

Review comment:
       shall we call it right after when `sorted` becomes true?




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617348669



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int,
+    var latestFilePath: String)
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    if (currentWriterId.partitionValues != nextPartitionValues ||
+      currentWriterId.bucketId != nextBucketId) {
+      // See a new partition or bucket - write to a new partition dir (or a new bucket file).
+      updateCurrentWriterStatusInMap()
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    checkRecordsInFile(currentWriterId.partitionValues, currentWriterId.bucketId)
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status when a new writer is needed for writing row.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) {
+      if (!sortBased) {
+        // Update writer status in concurrent writers map, because the writer is probably needed
+        // again later for writing other rows.

Review comment:
       Is it a performance concern that we only update the writer status when it's going to be put back into the map? What if we update the status per row?




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825371987


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137831/
   


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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824389128


   **[Test build #137749 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137749/testReport)** for PR 32198 at commit [`f8b466e`](https://github.com/apache/spark/commit/f8b466ede20007900c1e139c5c27c2158e7aecf4).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823981409


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137716/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823778062


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42242/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824826577


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137784/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r620009070



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -243,6 +273,52 @@ class DynamicPartitionDataWriter(
     statsTrackers.foreach(_.newFile(currentPath))
   }
 
+  /**
+   * Increase the file counter and open a new OutputWriter.
+   * This is used when number of records records exceeding limit.
+   *
+   * @param partitionValues the partition which all tuples being written by this `OutputWriter`
+   *                        belong to
+   * @param bucketId the bucket which all tuples being written by this `OutputWriter` belong to
+   */
+  protected def increaseFileCounter(

Review comment:
       ah, let's keep it as it is




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824507409


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137753/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824425593






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


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

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618795687



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None

Review comment:
       Got it. Thanks for the explanation.




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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820847632


   **[Test build #137451 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137451/testReport)** for PR 32198 at commit [`18f2851`](https://github.com/apache/spark/commit/18f285160bdca3c7958c5446c5d6fbcbef84b934).


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


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

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r619002468



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      if (!sorted) {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters,
+          s"Number of concurrent output file writers is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}")
+      } else {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1,
+          s"Number of output file writers after sort is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}")
+      }
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size >= concurrentOutputWriterSpec.maxWriters && !sorted) {
+        // Fall back to sort-based sequential writer mode.
+        sorted = true

Review comment:
       Just for future perfermance debug. Can we add a info log at this code place ?




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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618781460



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(

Review comment:
       @imback82 - I am hesitating to introduce assertion as this are on row execution path unless it's very necessary. Are you worried about `concurrentWriters.size > concurrentOutputWriterSpec.maxWriters`? Note this is the only place in file to add new keys into map (the only `.put()` call), and it has the check for size immediately at next line.




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


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

Posted by GitBox <gi...@apache.org>.
hvanhovell commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-828320490


   One more thing, how much does this improve the write? Local sorts before the write are typically not too bad if you look at the cycles spend during the write. A much bigger target here would be to properly interleave I/O and CPU operations. You sort of achieve that by having multiple writers, but it IMO feels like quite a big hammer. 


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-828539579


   > How do you avoid OOMs?
   
   Note the feature is designed to be disabled by default, and to be enabled case by case now. The fallback logic here is intended to avoid OOM when opening too many writers.


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824422127


   **[Test build #137753 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137753/testReport)** for PR 32198 at commit [`52c6160`](https://github.com/apache/spark/commit/52c6160fc017afba233478271ca69ea40e799ddb).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826755789


   **[Test build #137935 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137935/testReport)** for PR 32198 at commit [`2895837`](https://github.com/apache/spark/commit/2895837e8d80e371388a23207f3c897808e62a55).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820867956


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42026/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824442645


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42280/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824798440


   **[Test build #137784 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137784/testReport)** for PR 32198 at commit [`a4ed53b`](https://github.com/apache/spark/commit/a4ed53b702f1514de66a5696989abfc85e14a551).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824567051


   **[Test build #137784 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137784/testReport)** for PR 32198 at commit [`a4ed53b`](https://github.com/apache/spark/commit/a4ed53b702f1514de66a5696989abfc85e14a551).


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820886083


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42033/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820938807


   **[Test build #137451 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137451/testReport)** for PR 32198 at commit [`18f2851`](https://github.com/apache/spark/commit/18f285160bdca3c7958c5446c5d6fbcbef84b934).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  case class ConcurrentOutputWriterSpec(`


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


[GitHub] [spark] SparksFyz edited a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparksFyz edited a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-1062764141


   > > One more thing, how much does this improve the write? Local sorts before the write are typically not too bad if you look at the cycles spend during the write. A much bigger target here would be to properly interleave I/O and CPU operations. You sort of achieve that by having multiple writers, but it IMO feels like quite a big hammer.
   > 
   > I will add a benchmark for this as a followup.
   > 
   > IMHO how much this can improve thing is really depending on query shape (cardinality of dynamic partitions and buckets). In one environment, if most queries having low number of partitions and users set buckets relatively small, this feature can help more. If in another environment, query tends to write a lot of partitions and users set buckets quite large, this feature helps less. We do see benefit for improving query internally and people raised the request in spark dev as well.
   
   @c21 Hi~ Any link for benchmark? Thanks~


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824442645


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42280/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825151839


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42350/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618991351



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)

Review comment:
       Can we add a comment to explain it?




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618157080



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      newOutputWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))

Review comment:
       It's a new writer, the initial value of currentWriter should be null?




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823873066


   **[Test build #137714 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137714/testReport)** for PR 32198 at commit [`0442f05`](https://github.com/apache/spark/commit/0442f0506ee1cd694842480da2e2817ae0be04c9).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `abstract class BaseDynamicPartitionDataWriter(`
     * `class DynamicPartitionDataSingleWriter(`
     * `class DynamicPartitionDataConcurrentWriter(`


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618179109



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0

Review comment:
       @cloud-fan - it's reset in `newOutputWriter`. Though it's tricky, I think it makes sense that whenever a new output writer is needed (i.e. calling `newOutputWriter`), `recordsInFile` should be reset to 0 (so it's part of method in `newOutputWriter`), but not necessarily for `fileCounter` (so it's not part of method in `newOutputWriter`).




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618997526



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()

Review comment:
       How about `setupCurrentWriterUsingMap`? The main logic here is to set up the current writer before calling `writeRecord`




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824474170


   **[Test build #137749 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137749/testReport)** for PR 32198 at commit [`f8b466e`](https://github.com/apache/spark/commit/f8b466ede20007900c1e139c5c27c2158e7aecf4).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824439529


   Kubernetes integration test unable to build dist.
   
   exiting with code: 1
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42280/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827088743


   **[Test build #137970 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137970/testReport)** for PR 32198 at commit [`efe026c`](https://github.com/apache/spark/commit/efe026cc797db994787b6a556aee699d6a79d4af).


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


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

Posted by GitBox <gi...@apache.org>.
hvanhovell commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-828317277


   @c21 this doesn't do any sort of memory tracking right? How do you avoid OOMs?


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


[GitHub] [spark] SparksFyz edited a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparksFyz edited a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-1062764141


   > > One more thing, how much does this improve the write? Local sorts before the write are typically not too bad if you look at the cycles spend during the write. A much bigger target here would be to properly interleave I/O and CPU operations. You sort of achieve that by having multiple writers, but it IMO feels like quite a big hammer.
   > 
   > I will add a benchmark for this as a followup.
   > 
   > IMHO how much this can improve thing is really depending on query shape (cardinality of dynamic partitions and buckets). In one environment, if most queries having low number of partitions and users set buckets relatively small, this feature can help more. If in another environment, query tends to write a lot of partitions and users set buckets quite large, this feature helps less. We do see benefit for improving query internally and people raised the request in spark dev as well.
   
   @c21 Hi~ Any link for benchmark? Thanks~
   I am interested in how much does this improve for tuning the number to 2 for static partition write(single partition)


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

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618999574



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -243,6 +273,52 @@ class DynamicPartitionDataWriter(
     statsTrackers.foreach(_.newFile(currentPath))
   }
 
+  /**
+   * Increase the file counter and open a new OutputWriter.
+   * This is used when number of records records exceeding limit.
+   *
+   * @param partitionValues the partition which all tuples being written by this `OutputWriter`
+   *                        belong to
+   * @param bucketId the bucket which all tuples being written by this `OutputWriter` belong to
+   */
+  protected def increaseFileCounter(

Review comment:
       shall we put the `if (... recordsInFile >= description.maxRecordsPerFile)` in this method as well? Then the method can be named as `renewCurrentWriterIfTooManyRecords`




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617339760



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent output writer. */
+  private case class WriterIndex(

Review comment:
       ditto




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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827223646


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137970/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820951541


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137451/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617339760



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent output writer. */
+  private case class WriterIndex(

Review comment:
       ditto




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618158315



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      newOutputWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size > concurrentOutputWriterSpec.maxWriters &&
+        !sortBased) {
+        // Fall back to sort-based sequential writer mode.
+        sortBased = true
+      }
+    }
+  }
+
+  /**
+   * Clear the current writer status in map.
+   */
+  private def clearCurrentWriterStatus(): Unit = {
+    if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) {
+      val status = concurrentWriters(currentWriterId)

Review comment:
       shall we simply call `updateCurrentWriterStatusInMap` here?




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823980262


   **[Test build #137716 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137716/testReport)** for PR 32198 at commit [`b527e37`](https://github.com/apache/spark/commit/b527e371ba94eb5b8d25eb1dffb35ffd9331686d).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617321522



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala
##########
@@ -134,23 +134,20 @@ class BasicWriteTaskStatsTracker(hadoopConf: Configuration)
     partitions.append(partitionValues)
   }
 
-  override def newBucket(bucketId: Int): Unit = {
-    // currently unhandled
+  override def newFile(filePath: String): Unit = {
+    submittedFiles += filePath
+    numSubmittedFiles += 1
   }
 
-  override def newFile(filePath: String): Unit = {
-    statCurrentFile()
-    curFile = Some(filePath)
-    submittedFiles += 1
+  override def closeFile(filePath: String): Unit = {
+    getFileStats(filePath)
+    submittedFiles.remove(filePath)
   }
 
-  private def statCurrentFile(): Unit = {
-    curFile.foreach { path =>
-      getFileSize(path).foreach { len =>
-        numBytes += len
-        numFiles += 1
-      }
-      curFile = None
+  private def getFileStats(filePath: String): Unit = {

Review comment:
       seems it's not `getFileStats`, but `updateFileStats`




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618151334



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false

Review comment:
       nit: `sorted` seems better




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823829022


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42244/
   


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618810577



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,180 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(

Review comment:
       ok, updated to add add assertion before `put`. The assertion is different before and after sort.




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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618664024



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false

Review comment:
       @cloud-fan - updated.

##########
File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriter.scala
##########
@@ -84,4 +84,6 @@ private[avro] class AvroOutputWriter(
   }
 
   override def close(): Unit = recordWriter.close(context)
+
+  override def path(): String = path

Review comment:
       @cloud-fan - updated, thanks!

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      newOutputWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size > concurrentOutputWriterSpec.maxWriters &&
+        !sortBased) {
+        // Fall back to sort-based sequential writer mode.
+        sortBased = true
+      }
+    }
+  }
+
+  /**
+   * Clear the current writer status in map.
+   */
+  private def clearCurrentWriterStatus(): Unit = {
+    if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) {
+      val status = concurrentWriters(currentWriterId)

Review comment:
       @cloud-fan - updated now.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0

Review comment:
       @cloud-fan - sure, updated with `renewCurrentWriter`.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824629754


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42313/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827223646


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137970/
   


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618180197



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sortBased) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      newOutputWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size > concurrentOutputWriterSpec.maxWriters &&
+        !sortBased) {
+        // Fall back to sort-based sequential writer mode.
+        sortBased = true
+      }
+    }
+  }
+
+  /**
+   * Clear the current writer status in map.
+   */
+  private def clearCurrentWriterStatus(): Unit = {
+    if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) {
+      val status = concurrentWriters(currentWriterId)

Review comment:
       @cloud-fan - yes, you are right. Will updated.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826581183


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/42457/
   


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824483915


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137749/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824389128


   **[Test build #137749 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137749/testReport)** for PR 32198 at commit [`f8b466e`](https://github.com/apache/spark/commit/f8b466ede20007900c1e139c5c27c2158e7aecf4).


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820873113


   @HyukjinKwon - thanks for the heads up, updated.


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825256785


   **[Test build #137831 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137831/testReport)** for PR 32198 at commit [`17a80a6`](https://github.com/apache/spark/commit/17a80a6f02d2011f7dc6d9ee6c4a9843af63fa62).


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618159188



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)

Review comment:
       This looks a bit fragile as the map key is mutable. Can we always create a new `WriterIndex` instance when needed and make `WriterIndex` immutable?




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617333442



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -47,6 +48,7 @@ abstract class FileFormatDataWriter(
   protected val MAX_FILE_COUNTER: Int = 1000 * 1000
   protected val updatedPartitions: mutable.Set[String] = mutable.Set[String]()
   protected var currentWriter: OutputWriter = _

Review comment:
       It seems all `OutputWriter` implementations have a path string. Shall we simply add a `def path: String` in `OutputWriter`? Then we don't need the `currentPath`




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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r619979939



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -243,6 +273,52 @@ class DynamicPartitionDataWriter(
     statsTrackers.foreach(_.newFile(currentPath))
   }
 
+  /**
+   * Increase the file counter and open a new OutputWriter.
+   * This is used when number of records records exceeding limit.
+   *
+   * @param partitionValues the partition which all tuples being written by this `OutputWriter`
+   *                        belong to
+   * @param bucketId the bucket which all tuples being written by this `OutputWriter` belong to
+   */
+  protected def increaseFileCounter(

Review comment:
       In `DynamicPartitionDataConcurrentWriter.write`, I need this condition check to update writer in map as well (as the current writer is renewed). If we put the condition inside here, we need double check in `DynamicPartitionDataConcurrentWriter.write`. Shall we keep it as it is?




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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617137223



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

Review comment:
       Yes.




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


[GitHub] [spark] github-actions[bot] commented on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820872997


   **[Test build #754194290](https://github.com/c21/spark/actions/runs/754194290)** for PR 32198 at commit [`4b2801b`](https://github.com/c21/spark/commit/4b2801b9ccf5789c928d0105b04634e26924cf0d).


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r619978928



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -52,19 +53,35 @@ abstract class FileFormatDataWriter(
   protected val statsTrackers: Seq[WriteTaskStatsTracker] =
     description.statsTrackers.map(_.newTaskInstance())
 
-  protected def releaseResources(): Unit = {
+  /** Release resources of `currentWriter`. */
+  protected def releaseCurrentWriter(): Unit = {
     if (currentWriter != null) {
       try {
         currentWriter.close()
+        statsTrackers.foreach(_.closeFile(currentWriter.path()))
       } finally {
         currentWriter = null
       }
     }
   }
 
-  /** Writes a record */
+  /** Release all resources. */
+  protected def releaseResources(): Unit = {
+    // Call `releaseCurrentWriter()` by default, as this is the only resource to be released.
+    releaseCurrentWriter()
+  }
+
+  /** Writes a record. */
   def write(record: InternalRow): Unit

Review comment:
       We have [data source v2 writers depend on this and has a counter](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala#L390-L394), and this `write` is part of `DataWriter` API. Shall we change it later?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -243,6 +273,52 @@ class DynamicPartitionDataWriter(
     statsTrackers.foreach(_.newFile(currentPath))
   }
 
+  /**
+   * Increase the file counter and open a new OutputWriter.
+   * This is used when number of records records exceeding limit.
+   *
+   * @param partitionValues the partition which all tuples being written by this `OutputWriter`
+   *                        belong to
+   * @param bucketId the bucket which all tuples being written by this `OutputWriter` belong to
+   */
+  protected def increaseFileCounter(

Review comment:
       In `DynamicPartitionDataConcurrentWriter.write`, I need this condition check to update writer in map as well (as the current writer is renewed). If we put the condition inside here, we need double check in `DynamicPartitionDataConcurrentWriter.write`. Shall we keep it as it is?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)

Review comment:
       @cloud-fan - sure, added.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      if (!sorted) {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters,
+          s"Number of concurrent output file writers is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}")
+      } else {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1,
+          s"Number of output file writers after sort is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}")
+      }
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),

Review comment:
       @cloud-fan - updated.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    if (description.maxRecordsPerFile > 0 &&
+      recordsInFile >= description.maxRecordsPerFile) {
+      increaseFileCounter(currentWriterId.partitionValues, currentWriterId.bucketId)
+      // Update writer status in concurrent writers map, as a new writer is created.
+      updateCurrentWriterStatusInMap()
+    }
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sorted) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status in map.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    val status = concurrentWriters(currentWriterId)
+    status.outputWriter = currentWriter
+    status.recordsInFile = recordsInFile
+    status.fileCounter = fileCounter
+  }
+
+  /**
+   * Retrieve writer in map, or create a new writer if not exists.
+   */
+  private def retrieveWriterInMap(): Unit = {
+    if (concurrentWriters.contains(currentWriterId)) {
+      val status = concurrentWriters(currentWriterId)
+      currentWriter = status.outputWriter
+      recordsInFile = status.recordsInFile
+      fileCounter = status.fileCounter
+    } else {
+      fileCounter = 0
+      renewCurrentWriter(
+        currentWriterId.partitionValues,
+        currentWriterId.bucketId,
+        closeCurrentWriter = false)
+      if (!sorted) {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters,
+          s"Number of concurrent output file writers is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}")
+      } else {
+        assert(concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1,
+          s"Number of output file writers after sort is ${concurrentWriters.size} " +
+            s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}")
+      }
+      concurrentWriters.put(
+        WriterIndex(currentWriterId.partitionValues, currentWriterId.bucketId),
+        new WriterStatus(currentWriter, recordsInFile, fileCounter))
+      if (concurrentWriters.size >= concurrentOutputWriterSpec.maxWriters && !sorted) {
+        // Fall back to sort-based sequential writer mode.
+        sorted = true

Review comment:
       @ulysses-you - good callout. Updated with a info logging. I think it makes sense to account how many tasks have been fallbacked and show on UI, I will take that as a followup.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,189 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sorted: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    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 (currentWriter != null) {
+        if (!sorted) {
+          // Update writer status in concurrent writers map, because the writer is probably needed
+          // again later for writing other rows.
+          updateCurrentWriterStatusInMap()
+        } else {
+          // Remove writer status in concurrent writers map and release current writer resource,
+          // because the writer is not needed any more.
+          concurrentWriters.remove(currentWriterId)
+          releaseCurrentWriter()
+        }
+      }
+
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()

Review comment:
       @cloud-fan - updated.

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
##########
@@ -1219,4 +1219,38 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with
       }
     }
   }
+
+  test("SPARK-26164: Allow concurrent writers for multiple partitions and buckets") {
+    withTable("t1", "t2") {
+      val df = spark.range(200).map(_ => {
+        val n = scala.util.Random.nextInt

Review comment:
       @cloud-fan - good call. Updated.




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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824507409


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137753/
   


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820886067






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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617869815



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -3150,6 +3150,14 @@ object SQLConf {
     .booleanConf
     .createWithDefault(false)
 
+  val MAX_CONCURRENT_OUTPUT_WRITERS = buildConf("spark.sql.maxConcurrentOutputWriters")

Review comment:
       @cloud-fan - updated.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -47,6 +48,7 @@ abstract class FileFormatDataWriter(
   protected val MAX_FILE_COUNTER: Int = 1000 * 1000
   protected val updatedPartitions: mutable.Set[String] = mutable.Set[String]()
   protected var currentWriter: OutputWriter = _

Review comment:
       @cloud-fan - makes sense. Af first place I was hesitating to make broader change of interface `OutputWriter`. But updated now.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 WriterStatus(

Review comment:
       @cloud-fan - updated.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int,
+    var latestFilePath: String)
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null

Review comment:
       @cloud-fan - yes it should be not null here as it indicates the last active writer. `currentWriter` is always put inside `concurrentWriters` in `retrieveWriterInMap()`. So it should be closed below when iterating `concurrentWriters.values`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -56,6 +58,7 @@ abstract class FileFormatDataWriter(
     if (currentWriter != null) {

Review comment:
       @cloud-fan - yes I agree with that looks pretty weird. Good idea and updated.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +320,182 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
-    } else 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, true)
+    } else {
+      checkRecordsInFile(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int,
+    var latestFilePath: String)
 
-      newOutputWriter(currentPartitionValues, currentBucketId)
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)
+
+  /**
+   * Release resources for all concurrent output writers.
+   */
+  override protected def releaseResources(): Unit = {
+    currentWriter = null
+    concurrentWriters.values.foreach(status => {
+      if (status.outputWriter != null) {
+        try {
+          status.outputWriter.close()
+        } finally {
+          status.outputWriter = null
+        }
+      }
+    })
+    concurrentWriters.clear()
+  }
+
+  override def write(record: InternalRow): Unit = {
+    val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None
+    val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None
+
+    if (currentWriterId.partitionValues != nextPartitionValues ||
+      currentWriterId.bucketId != nextBucketId) {
+      // See a new partition or bucket - write to a new partition dir (or a new bucket file).
+      updateCurrentWriterStatusInMap()
+      if (isBucketed) {
+        currentWriterId.bucketId = nextBucketId
+      }
+      if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) {
+        currentWriterId.partitionValues = Some(nextPartitionValues.get.copy())
+        if (!concurrentWriters.contains(currentWriterId)) {
+          statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get))
+        }
+      }
+      retrieveWriterInMap()
     }
-    val outputRow = getOutputRow(record)
-    currentWriter.write(outputRow)
-    statsTrackers.foreach(_.newRow(outputRow))
-    recordsInFile += 1
+
+    checkRecordsInFile(currentWriterId.partitionValues, currentWriterId.bucketId)
+    writeRecord(record)
+  }
+
+  /**
+   * Write iterator of records with concurrent writers.
+   */
+  def writeWithIterator(iterator: Iterator[InternalRow]): Unit = {
+    while (iterator.hasNext && !sortBased) {
+      write(iterator.next())
+    }
+
+    if (iterator.hasNext) {
+      clearCurrentWriterStatus()
+      val sorter = concurrentOutputWriterSpec.createSorter()
+      val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]])
+      while (sortIterator.hasNext) {
+        write(sortIterator.next())
+      }
+    }
+  }
+
+  /**
+   * Update current writer status when a new writer is needed for writing row.
+   */
+  private def updateCurrentWriterStatusInMap(): Unit = {
+    if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) {
+      if (!sortBased) {
+        // Update writer status in concurrent writers map, because the writer is probably needed
+        // again later for writing other rows.

Review comment:
       I want to avoid extra computation (e.g. lookup in hashmap) if it is unnecessary. I don't see a benefit to update status in map per row, as anyway the current writer's status is captured in `(currentWriter, recordsInFile, fileCounter)` already.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
##########
@@ -273,14 +289,25 @@ object FileFormatWriter extends Logging {
       } else if (description.partitionColumns.isEmpty && description.bucketIdExpression.isEmpty) {
         new SingleDirectoryDataWriter(description, taskAttemptContext, committer)
       } else {
-        new DynamicPartitionDataWriter(description, taskAttemptContext, committer)
+        concurrentOutputWriterSpec match {
+          case Some(spec) =>
+            new DynamicPartitionDataConcurrentWriter(
+              description, taskAttemptContext, committer, spec)
+          case _ =>
+            new DynamicPartitionDataSingleWriter(description, taskAttemptContext, committer)
+        }
       }
 
     try {
       Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
         // Execute the task to write rows out and commit the task.
-        while (iterator.hasNext) {
-          dataWriter.write(iterator.next())
+        dataWriter match {
+          case w: DynamicPartitionDataConcurrentWriter =>
+            w.writeWithIterator(iterator)

Review comment:
       @cloud-fan - wondering what's the benefit of doing it? Updated anyway now.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala
##########
@@ -134,23 +134,20 @@ class BasicWriteTaskStatsTracker(hadoopConf: Configuration)
     partitions.append(partitionValues)
   }
 
-  override def newBucket(bucketId: Int): Unit = {
-    // currently unhandled
+  override def newFile(filePath: String): Unit = {
+    submittedFiles += filePath
+    numSubmittedFiles += 1
   }
 
-  override def newFile(filePath: String): Unit = {
-    statCurrentFile()
-    curFile = Some(filePath)
-    submittedFiles += 1
+  override def closeFile(filePath: String): Unit = {
+    getFileStats(filePath)
+    submittedFiles.remove(filePath)
   }
 
-  private def statCurrentFile(): Unit = {
-    curFile.foreach { path =>
-      getFileSize(path).foreach { len =>
-        numBytes += len
-        numFiles += 1
-      }
-      curFile = None
+  private def getFileStats(filePath: String): Unit = {

Review comment:
       @cloud-fan - updated.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824506522


   **[Test build #137753 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137753/testReport)** for PR 32198 at commit [`52c6160`](https://github.com/apache/spark/commit/52c6160fc017afba233478271ca69ea40e799ddb).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
c21 commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r619978928



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -52,19 +53,35 @@ abstract class FileFormatDataWriter(
   protected val statsTrackers: Seq[WriteTaskStatsTracker] =
     description.statsTrackers.map(_.newTaskInstance())
 
-  protected def releaseResources(): Unit = {
+  /** Release resources of `currentWriter`. */
+  protected def releaseCurrentWriter(): Unit = {
     if (currentWriter != null) {
       try {
         currentWriter.close()
+        statsTrackers.foreach(_.closeFile(currentWriter.path()))
       } finally {
         currentWriter = null
       }
     }
   }
 
-  /** Writes a record */
+  /** Release all resources. */
+  protected def releaseResources(): Unit = {
+    // Call `releaseCurrentWriter()` by default, as this is the only resource to be released.
+    releaseCurrentWriter()
+  }
+
+  /** Writes a record. */
   def write(record: InternalRow): Unit

Review comment:
       We have [data source v2 writers depend on this and has a counter](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala#L390-L394), and this `write` is part of `DataWriter` API. Shall we change it later?




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826542139


   **[Test build #137935 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137935/testReport)** for PR 32198 at commit [`2895837`](https://github.com/apache/spark/commit/2895837e8d80e371388a23207f3c897808e62a55).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-826542139


   **[Test build #137935 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137935/testReport)** for PR 32198 at commit [`2895837`](https://github.com/apache/spark/commit/2895837e8d80e371388a23207f3c897808e62a55).


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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-820884919


   **[Test build #137459 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137459/testReport)** for PR 32198 at commit [`4b2801b`](https://github.com/apache/spark/commit/4b2801b9ccf5789c928d0105b04634e26924cf0d).


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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-823776534






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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r620083683



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -243,6 +273,52 @@ class DynamicPartitionDataWriter(
     statsTrackers.foreach(_.newFile(currentPath))
   }
 
+  /**
+   * Increase the file counter and open a new OutputWriter.
+   * This is used when number of records records exceeding limit.
+   *
+   * @param partitionValues the partition which all tuples being written by this `OutputWriter`
+   *                        belong to
+   * @param bucketId the bucket which all tuples being written by this `OutputWriter` belong to
+   */
+  protected def increaseFileCounter(

Review comment:
       let's still rename it: `renewCurrentWriterForTooManyRecords`




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618159188



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)

Review comment:
       This looks a bit fragile as the map key is mutable. Can we always create a new `WriterIndex` when needed and make `WriterIndex` immutable.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825362420


   **[Test build #137831 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137831/testReport)** for PR 32198 at commit [`17a80a6`](https://github.com/apache/spark/commit/17a80a6f02d2011f7dc6d9ee6c4a9843af63fa62).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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


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

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #32198:
URL: https://github.com/apache/spark/pull/32198


   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-827124409


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42490/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r616732203



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

Review comment:
       does it mean we can have `limit + 1` writers at most?




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r618159188



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala
##########
@@ -255,25 +331,184 @@ class DynamicPartitionDataWriter(
       }
       if (isBucketed) {
         currentBucketId = nextBucketId
-        statsTrackers.foreach(_.newBucket(currentBucketId.get))
       }
 
       fileCounter = 0
-      newOutputWriter(currentPartitionValues, currentBucketId)
+      newOutputWriter(currentPartitionValues, currentBucketId, true)
     } else 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")
+      increaseFileCounter(currentPartitionValues, currentBucketId)
+    }
+    writeRecord(record)
+  }
+}
+
+/**
+ * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened
+ * for writing.
+ *
+ * The process has the following steps:
+ *  - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all
+ *            writers opened and write rows one by one.
+ *  - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or
+ *            bucket column(s). Write rows one by one, and eagerly close the writer when finishing
+ *            each partition and/or bucket.
+ *
+ * Caller is expected to call `writeWithIterator()` instead of `write()` to write records.
+ */
+class DynamicPartitionDataConcurrentWriter(
+    description: WriteJobDescription,
+    taskAttemptContext: TaskAttemptContext,
+    committer: FileCommitProtocol,
+    concurrentOutputWriterSpec: ConcurrentOutputWriterSpec)
+  extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer) {
+
+  /** Wrapper class to index a unique concurrent 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 class WriterStatus(
+    var outputWriter: OutputWriter,
+    var recordsInFile: Long,
+    var fileCounter: Int)
+
+  /**
+   * State to indicate if we are falling back to sort-based writer.
+   * Because we first try to use concurrent writers, its initial value is false.
+   */
+  private var sortBased: Boolean = false
+  private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]()
+  private val currentWriterId = WriterIndex(None, None)

Review comment:
       This looks a bit fragile as the map key is mutable. Can we always create a new `WriterIndex` instance when needed and make `WriterIndex` immutable.




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r619001681



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala
##########
@@ -1219,4 +1219,38 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with
       }
     }
   }
+
+  test("SPARK-26164: Allow concurrent writers for multiple partitions and buckets") {
+    withTable("t1", "t2") {
+      val df = spark.range(200).map(_ => {
+        val n = scala.util.Random.nextInt

Review comment:
       can we use a fixed seed in the test? Otherwise there is a small possibility that the distinct values are less than 3 and the fallback test doesn't trigger.




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


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

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825144912






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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825256785


   **[Test build #137831 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137831/testReport)** for PR 32198 at commit [`17a80a6`](https://github.com/apache/spark/commit/17a80a6f02d2011f7dc6d9ee6c4a9843af63fa62).


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


[GitHub] [spark] SparkQA removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-825116985


   **[Test build #137821 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/137821/testReport)** for PR 32198 at commit [`1e8d447`](https://github.com/apache/spark/commit/1e8d447e8c8d6a4cd9954a719d8a55907bcb3e5d).


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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32198: [SPARK-26164][SQL] Allow concurrent writers for writing dynamic partitions and bucket table

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824826577


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/137784/
   


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


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

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32198:
URL: https://github.com/apache/spark/pull/32198#issuecomment-824425619


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/42276/
   


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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617343611



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala
##########
@@ -273,14 +289,25 @@ object FileFormatWriter extends Logging {
       } else if (description.partitionColumns.isEmpty && description.bucketIdExpression.isEmpty) {
         new SingleDirectoryDataWriter(description, taskAttemptContext, committer)
       } else {
-        new DynamicPartitionDataWriter(description, taskAttemptContext, committer)
+        concurrentOutputWriterSpec match {
+          case Some(spec) =>
+            new DynamicPartitionDataConcurrentWriter(
+              description, taskAttemptContext, committer, spec)
+          case _ =>
+            new DynamicPartitionDataSingleWriter(description, taskAttemptContext, committer)
+        }
       }
 
     try {
       Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
         // Execute the task to write rows out and commit the task.
-        while (iterator.hasNext) {
-          dataWriter.write(iterator.next())
+        dataWriter match {
+          case w: DynamicPartitionDataConcurrentWriter =>
+            w.writeWithIterator(iterator)

Review comment:
       We can make it an API in the base class, which by default just do
   ```
   while (iterator.hasNext) {
     write(iterator.next())
   }
   ```




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


[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

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #32198:
URL: https://github.com/apache/spark/pull/32198#discussion_r617318878



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -3150,6 +3150,14 @@ object SQLConf {
     .booleanConf
     .createWithDefault(false)
 
+  val MAX_CONCURRENT_OUTPUT_WRITERS = buildConf("spark.sql.maxConcurrentOutputWriters")

Review comment:
       `maxConcurrentOutputFileWriters`? To indicate it's for file source only.




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