-
Notifications
You must be signed in to change notification settings - Fork 28.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-19085][SQL] cleanup OutputWriterFactory and OutputWriter #16479
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -64,18 +64,18 @@ object FileFormatWriter extends Logging { | |
val outputWriterFactory: OutputWriterFactory, | ||
val allColumns: Seq[Attribute], | ||
val partitionColumns: Seq[Attribute], | ||
val nonPartitionColumns: Seq[Attribute], | ||
val dataColumns: Seq[Attribute], | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. rename |
||
val bucketSpec: Option[BucketSpec], | ||
val path: String, | ||
val customPartitionLocations: Map[TablePartitionSpec, String], | ||
val maxRecordsPerFile: Long) | ||
extends Serializable { | ||
|
||
assert(AttributeSet(allColumns) == AttributeSet(partitionColumns ++ nonPartitionColumns), | ||
assert(AttributeSet(allColumns) == AttributeSet(partitionColumns ++ dataColumns), | ||
s""" | ||
|All columns: ${allColumns.mkString(", ")} | ||
|Partition columns: ${partitionColumns.mkString(", ")} | ||
|Non-partition columns: ${nonPartitionColumns.mkString(", ")} | ||
|Data columns: ${dataColumns.mkString(", ")} | ||
""".stripMargin) | ||
} | ||
|
||
|
@@ -120,7 +120,7 @@ object FileFormatWriter extends Logging { | |
outputWriterFactory = outputWriterFactory, | ||
allColumns = queryExecution.logical.output, | ||
partitionColumns = partitionColumns, | ||
nonPartitionColumns = dataColumns, | ||
dataColumns = dataColumns, | ||
bucketSpec = bucketSpec, | ||
path = outputSpec.outputPath, | ||
customPartitionLocations = outputSpec.customPartitionLocations, | ||
|
@@ -246,9 +246,8 @@ object FileFormatWriter extends Logging { | |
|
||
currentWriter = description.outputWriterFactory.newInstance( | ||
path = tmpFilePath, | ||
dataSchema = description.nonPartitionColumns.toStructType, | ||
dataSchema = description.dataColumns.toStructType, | ||
context = taskAttemptContext) | ||
currentWriter.initConverter(dataSchema = description.nonPartitionColumns.toStructType) | ||
} | ||
|
||
override def execute(iter: Iterator[InternalRow]): Set[String] = { | ||
|
@@ -267,7 +266,7 @@ object FileFormatWriter extends Logging { | |
} | ||
|
||
val internalRow = iter.next() | ||
currentWriter.writeInternal(internalRow) | ||
currentWriter.write(internalRow) | ||
recordsInFile += 1 | ||
} | ||
releaseResources() | ||
|
@@ -364,9 +363,8 @@ object FileFormatWriter extends Logging { | |
|
||
currentWriter = description.outputWriterFactory.newInstance( | ||
path = path, | ||
dataSchema = description.nonPartitionColumns.toStructType, | ||
dataSchema = description.dataColumns.toStructType, | ||
context = taskAttemptContext) | ||
currentWriter.initConverter(description.nonPartitionColumns.toStructType) | ||
} | ||
|
||
override def execute(iter: Iterator[InternalRow]): Set[String] = { | ||
|
@@ -383,7 +381,7 @@ object FileFormatWriter extends Logging { | |
|
||
// Returns the data columns to be written given an input row | ||
val getOutputRow = UnsafeProjection.create( | ||
description.nonPartitionColumns, description.allColumns) | ||
description.dataColumns, description.allColumns) | ||
|
||
// Returns the partition path given a partition key. | ||
val getPartitionStringFunc = UnsafeProjection.create( | ||
|
@@ -392,7 +390,7 @@ object FileFormatWriter extends Logging { | |
// Sorts the data before write, so that we only need one writer at the same time. | ||
val sorter = new UnsafeKVExternalSorter( | ||
sortingKeySchema, | ||
StructType.fromAttributes(description.nonPartitionColumns), | ||
StructType.fromAttributes(description.dataColumns), | ||
SparkEnv.get.blockManager, | ||
SparkEnv.get.serializerManager, | ||
TaskContext.get().taskMemoryManager().pageSizeBytes, | ||
|
@@ -448,7 +446,7 @@ object FileFormatWriter extends Logging { | |
newOutputWriter(currentKey, getPartitionStringFunc, fileCounter) | ||
} | ||
|
||
currentWriter.writeInternal(sortedIterator.getValue) | ||
currentWriter.write(sortedIterator.getValue) | ||
recordsInFile += 1 | ||
} | ||
releaseResources() | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -47,19 +47,6 @@ abstract class OutputWriterFactory extends Serializable { | |
path: String, | ||
dataSchema: StructType, | ||
context: TaskAttemptContext): OutputWriter | ||
|
||
/** | ||
* Returns a new instance of [[OutputWriter]] that will write data to the given path. | ||
* This method gets called by each task on executor to write InternalRows to | ||
* format-specific files. Compared to the other `newInstance()`, this is a newer API that | ||
* passes only the path that the writer must write to. The writer must write to the exact path | ||
* and not modify it (do not add subdirectories, extensions, etc.). All other | ||
* file-format-specific information needed to create the writer must be passed | ||
* through the [[OutputWriterFactory]] implementation. | ||
*/ | ||
def newWriter(path: String): OutputWriter = { | ||
throw new UnsupportedOperationException("newInstance with just path not supported") | ||
} | ||
} | ||
|
||
|
||
|
@@ -74,22 +61,11 @@ abstract class OutputWriter { | |
* Persists a single row. Invoked on the executor side. When writing to dynamically partitioned | ||
* tables, dynamic partition columns are not included in rows to be written. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The usage of this function was removed in https://github.com/apache/spark/pull/15710/files I think it is safe to remove it. |
||
*/ | ||
def write(row: Row): Unit | ||
def write(row: InternalRow): Unit | ||
|
||
/** | ||
* Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before | ||
* the task output is committed. | ||
*/ | ||
def close(): Unit | ||
|
||
private var converter: InternalRow => Row = _ | ||
|
||
protected[sql] def initConverter(dataSchema: StructType) = { | ||
converter = | ||
CatalystTypeConverters.createToScalaConverter(dataSchema).asInstanceOf[InternalRow => Row] | ||
} | ||
|
||
protected[sql] def writeInternal(row: InternalRow): Unit = { | ||
write(converter(row)) | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LibSVMFileFormat.verifySchema
is only called in thebuildReader
, but this is the write path, right?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok I added the verification.