Github user nongli commented on a diff in the pull request:
https://github.com/apache/spark/pull/10498#discussion_r49002988
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala
---
@@ -335,6 +339,121 @@ private[sql] class DynamicPartitionWriterContainer(
val partitionName = Literal(c.name + "=") :: str :: Nil
if (i == 0) partitionName else Literal(Path.SEPARATOR) ::
partitionName
}
+ }
+
+ private def getBucketIdFromKey(key: InternalRow): Option[Int] = {
+ if (bucketSpec.isDefined) {
+ Some(key.getInt(partitionColumns.length))
+ } else {
+ None
+ }
+ }
+
+ private def sameBucket(key1: UnsafeRow, key2: UnsafeRow): Boolean = {
+ val bucketIdIndex = partitionColumns.length
+ if (key1.getInt(bucketIdIndex) != key2.getInt(bucketIdIndex)) {
+ false
+ } else {
+ var i = partitionColumns.length - 1
+ while (i >= 0) {
+ val dt = partitionColumns(i).dataType
+ if (key1.get(i, dt) != key2.get(i, dt)) return false
+ i -= 1
+ }
+ true
+ }
+ }
+
+ private def sortBasedWrite(
+ sorter: UnsafeKVExternalSorter,
+ iterator: Iterator[InternalRow],
+ getSortingKey: UnsafeProjection,
+ getOutputRow: UnsafeProjection,
+ getPartitionString: UnsafeProjection,
+ outputWriters: java.util.HashMap[InternalRow, OutputWriter]): Unit =
{
+ while (iterator.hasNext) {
+ val currentRow = iterator.next()
+ sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow))
+ }
+
+ logInfo(s"Sorting complete. Writing out partition files one at a
time.")
+
+ val needNewWriter: (UnsafeRow, UnsafeRow) => Boolean = if
(sortColumns.isEmpty) {
+ (key1, key2) => key1 != key2
+ } else {
+ (key1, key2) => key1 == null || !sameBucket(key1, key2)
+ }
+
+ val sortedIterator = sorter.sortedIterator()
+ var currentKey: UnsafeRow = null
+ var currentWriter: OutputWriter = null
+ try {
+ while (sortedIterator.next()) {
+ if (needNewWriter(currentKey, sortedIterator.getKey)) {
+ if (currentWriter != null) {
+ currentWriter.close()
+ }
+ currentKey = sortedIterator.getKey.copy()
+ logDebug(s"Writing partition: $currentKey")
+
+ // Either use an existing file from before, or open a new one.
+ currentWriter = outputWriters.remove(currentKey)
+ if (currentWriter == null) {
+ currentWriter = newOutputWriter(currentKey, getPartitionString)
+ }
+ }
+
+ currentWriter.writeInternal(sortedIterator.getValue)
+ }
+ } finally {
+ if (currentWriter != null) { currentWriter.close() }
+ }
+ }
+
+ /**
+ * Open and returns a new OutputWriter given a partition key and
optional bucket id.
+ * If bucket id is specified, we will append it to the end of the file
name, but before the
+ * file extension, e.g.
part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet
+ */
+ private def newOutputWriter(
+ key: InternalRow,
+ getPartitionString: UnsafeProjection): OutputWriter = {
+ val configuration = taskAttemptContext.getConfiguration
+ val path = if (partitionColumns.nonEmpty) {
+ val partitionPath = getPartitionString(key).getString(0)
+ configuration.set(
+ "spark.sql.sources.output.path", new Path(outputPath,
partitionPath).toString)
+ new Path(getWorkPath, partitionPath).toString
+ } else {
+ configuration.set("spark.sql.sources.output.path", outputPath)
+ getWorkPath
+ }
+ val bucketId = getBucketIdFromKey(key)
+ val newWriter = super.newOutputWriter(path, bucketId)
+ newWriter.initConverter(dataSchema)
+ newWriter
+ }
+
+ def writeRows(taskContext: TaskContext, iterator:
Iterator[InternalRow]): Unit = {
+ val outputWriters = new java.util.HashMap[InternalRow, OutputWriter]
+ executorSideSetup(taskContext)
+
+ var outputWritersCleared = false
+
+ // We should first sort by partition columns, then bucket id, and
finally sorting columns.
+ val getSortingKey =
+ UnsafeProjection.create(partitionColumns ++ bucketIdExpression ++
sortColumns, inputSchema)
+
+ val sortingKeySchema = if (bucketSpec.isEmpty) {
+ StructType.fromAttributes(partitionColumns)
+ } else { // If it's bucketed, we should also consider bucket id as
part of the key.
+ val fields = StructType.fromAttributes(partitionColumns)
+ .add("bucketId", IntegerType, nullable = false) ++
StructType.fromAttributes(sortColumns)
+ StructType(fields)
+ }
--- End diff --
can this be simplifed to
sortingKeySchema = // what you have
val getSortingKey = UnsafeProject.create(sortingKeySchema)
?
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]