-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18042][SQL] OutputWriter should expose file path written #15580
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 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 |
|---|---|---|
|
|
@@ -35,7 +35,7 @@ private[parquet] class ParquetOptions( | |
| * Compression codec to use. By default use the value specified in SQLConf. | ||
| * Acceptable values are defined in [[shortParquetCompressionCodecNames]]. | ||
| */ | ||
| val compressionCodec: String = { | ||
| val compressionCodecClassName: String = { | ||
|
Member
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. @rxin This is a super minor but there are the same options to potentially rename in Also, I'd like to note, just in case, that the value in this is actually not the class name (it's something like |
||
| val codecName = parameters.getOrElse("compression", sqlConf.parquetCompressionCodec).toLowerCase | ||
| if (!shortParquetCompressionCodecNames.contains(codecName)) { | ||
| val availableCodecs = shortParquetCompressionCodecNames.keys.map(_.toLowerCase) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path | |
| import org.apache.hadoop.mapreduce._ | ||
| import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl | ||
| import org.apache.parquet.hadoop.{ParquetOutputFormat, ParquetRecordWriter} | ||
| import org.apache.parquet.hadoop.codec.CodecConfig | ||
| import org.apache.parquet.hadoop.util.ContextUtil | ||
|
|
||
| import org.apache.spark.sql.Row | ||
|
|
@@ -80,15 +81,15 @@ private[parquet] class ParquetOutputWriterFactory( | |
| sqlConf.writeLegacyParquetFormat.toString) | ||
|
|
||
| // Sets compression scheme | ||
| conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) | ||
| conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) | ||
| new SerializableConfiguration(conf) | ||
| } | ||
|
|
||
| /** | ||
| * Returns a [[OutputWriter]] that writes data to the give path without using | ||
| * [[OutputCommitter]]. | ||
| */ | ||
| override def newWriter(path: String): OutputWriter = new OutputWriter { | ||
| override def newWriter(path1: String): OutputWriter = new OutputWriter { | ||
|
Contributor
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. how about
Contributor
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. or maybe we can create a class for this |
||
|
|
||
| // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter | ||
| private val hadoopTaskAttemptId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) | ||
|
|
@@ -98,6 +99,8 @@ private[parquet] class ParquetOutputWriterFactory( | |
| // Instance of ParquetRecordWriter that does not use OutputCommitter | ||
| private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) | ||
|
|
||
| override def path: String = path1 | ||
|
|
||
| override def write(row: Row): Unit = { | ||
| throw new UnsupportedOperationException("call writeInternal") | ||
| } | ||
|
|
@@ -140,16 +143,17 @@ private[parquet] class ParquetOutputWriter( | |
| context: TaskAttemptContext) | ||
| extends OutputWriter { | ||
|
|
||
| override val path: String = { | ||
| val filename = fileNamePrefix + CodecConfig.from(context).getCodec.getExtension + ".parquet" | ||
| new Path(stagingDir, filename).toString | ||
| } | ||
|
|
||
| private val recordWriter: RecordWriter[Void, InternalRow] = { | ||
| val outputFormat = { | ||
| new ParquetOutputFormat[InternalRow]() { | ||
| override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { | ||
| new Path(stagingDir, fileNamePrefix + extension) | ||
| } | ||
| new ParquetOutputFormat[InternalRow]() { | ||
| override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { | ||
|
Contributor
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. now we never use the
Contributor
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. yes
Contributor
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. I will add some documentation in my next pull request. |
||
| new Path(path) | ||
| } | ||
| } | ||
|
|
||
| outputFormat.getRecordWriter(context) | ||
| }.getRecordWriter(context) | ||
| } | ||
|
|
||
| override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") | ||
|
|
||
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.
fullOutputPath?