-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23457][SQL] Register task completion listeners first in ParquetFileFormat #20619
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
Changes from 1 commit
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 |
|---|---|---|
|
|
@@ -395,16 +395,19 @@ class ParquetFileFormat | |
| ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get) | ||
| } | ||
| val taskContext = Option(TaskContext.get()) | ||
| val parquetReader = if (enableVectorizedReader) { | ||
| val iter = if (enableVectorizedReader) { | ||
| val vectorizedReader = new VectorizedParquetRecordReader( | ||
| convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) | ||
| val recordReaderIterator = new RecordReaderIterator(vectorizedReader) | ||
| // Register a task completion lister before `initalization`. | ||
| taskContext.foreach(_.addTaskCompletionListener(_ => recordReaderIterator.close())) | ||
| vectorizedReader.initialize(split, hadoopAttemptContext) | ||
| logDebug(s"Appending $partitionSchema ${file.partitionValues}") | ||
| vectorizedReader.initBatch(partitionSchema, file.partitionValues) | ||
| if (returningBatch) { | ||
| vectorizedReader.enableReturningBatches() | ||
| } | ||
| vectorizedReader | ||
| recordReaderIterator | ||
| } else { | ||
| logDebug(s"Falling back to parquet-mr") | ||
| // ParquetRecordReader returns UnsafeRow | ||
|
|
@@ -414,16 +417,16 @@ class ParquetFileFormat | |
| } else { | ||
| new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz)) | ||
| } | ||
| val recordReaderIterator = new RecordReaderIterator(reader) | ||
| // Register a task completion lister before `initalization`. | ||
| taskContext.foreach(_.addTaskCompletionListener(_ => recordReaderIterator.close())) | ||
| reader.initialize(split, hadoopAttemptContext) | ||
| reader | ||
| recordReaderIterator | ||
| } | ||
|
|
||
| val iter = new RecordReaderIterator(parquetReader) | ||
| taskContext.foreach(_.addTaskCompletionListener(_ => iter.close())) | ||
|
Member
Author
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. According to the reported leakage, this is too late. |
||
|
|
||
| // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. | ||
| if (parquetReader.isInstanceOf[VectorizedParquetRecordReader] && | ||
| enableVectorizedReader) { | ||
| if (enableVectorizedReader) { | ||
|
||
| iter.asInstanceOf[Iterator[InternalRow]] | ||
| } else { | ||
| val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes | ||
|
|
||
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.
could
new VectorizedParquetRecordReaderornew RecordReaderIteratorfail?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.
Those constructors didn't look heavy to me.
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