-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-15474][SQL] Write and read back non-emtpy schema with empty dataframe #19571
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 |
---|---|---|
|
@@ -32,7 +32,7 @@ import org.apache.hadoop.io.{NullWritable, Writable} | |
import org.apache.hadoop.mapred.{JobConf, OutputFormat => MapRedOutputFormat, RecordWriter, Reporter} | ||
import org.apache.hadoop.mapreduce._ | ||
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit} | ||
import org.apache.orc.OrcConf.COMPRESS | ||
import org.apache.orc.OrcConf.{COMPRESS, MAPRED_OUTPUT_SCHEMA} | ||
|
||
import org.apache.spark.TaskContext | ||
import org.apache.spark.sql.SparkSession | ||
|
@@ -58,10 +58,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable | |
sparkSession: SparkSession, | ||
options: Map[String, String], | ||
files: Seq[FileStatus]): Option[StructType] = { | ||
OrcFileOperator.readSchema( | ||
files.map(_.getPath.toString), | ||
Some(sparkSession.sessionState.newHadoopConf()) | ||
) | ||
org.apache.spark.sql.execution.datasources.orc.OrcFileFormat.readSchema(sparkSession, files) | ||
} | ||
|
||
override def prepareWrite( | ||
|
@@ -73,6 +70,10 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable | |
|
||
val configuration = job.getConfiguration | ||
|
||
configuration.set( | ||
MAPRED_OUTPUT_SCHEMA.getAttribute, | ||
org.apache.spark.sql.execution.datasources.orc.OrcFileFormat.getSchemaString(dataSchema)) | ||
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. Do we always need to set this? 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. This is the correct schema to be written. |
||
|
||
configuration.set(COMPRESS.getAttribute, orcOptions.compressionCodec) | ||
configuration match { | ||
case conf: JobConf => | ||
|
@@ -252,6 +253,12 @@ private[orc] class OrcOutputWriter( | |
override def close(): Unit = { | ||
if (recordWriterInstantiated) { | ||
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. Btw, according to the existing comment, seems that we can simply remove 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. In this PR, I'm focusing on emtpy file. We will replace the whole writer and reader with ORC 1.4.1 eventually. The newly added test case in this PR will make us to transit safely. |
||
recordWriter.close(Reporter.NULL) | ||
} else { | ||
// SPARK-15474 Write empty orc file with correct schema | ||
val conf = context.getConfiguration() | ||
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. Looks like the behavior to skip creating an empty file if no rows are written is deliberate. Is there any impact to current behavior? 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. Previously, only ORC does. So, it creates more issues like SPARK-22258 (#19477) and SPARK-21762. This is consistent with the other data sources like Parquet. |
||
val writer = org.apache.orc.OrcFile.createWriter( | ||
new Path(path), org.apache.orc.mapred.OrcOutputFormat.buildOptions(conf)) | ||
new org.apache.orc.mapreduce.OrcMapreduceRecordWriter(writer).close(context) | ||
} | ||
} | ||
} | ||
|
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.
I am not sure of this one too. This looks a complete rewrite of
org.apache.spark.sql.hive.orc.OrcFileOperator.readSchema
.. Is this change required to fix this issue?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.
Yes. It's intentional. OrcFileOperator will be replaced later completely. I made this PR as small as possible for review.