-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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-14387][SPARK-16628][SPARK-18355][SQL] Use Spark schema to read ORC table instead of ORC file schema #19470
Conversation
Test build #82620 has finished for PR 19470 at commit
|
Hi, @gatorsmile and @cloud-fan . |
Could you create test cases with the different schemas between files and hive metastore. |
Thank you for review, @gatorsmile . Here, this PR is focusing on missing-columns scenario after |
I remember we previously hit multiple issues due to the schema difference between the actual orc-file schema and the metastore schema. Just ensure it still exists and it does not make the current support worse. |
Ya, that was my question, too.
For me, HiveMetastore schema is the only valid one in Apache Spark. |
To be clear, I'll file another JIRA about ORC status on mismatched column orders. |
Hi, @gatorsmile .
Based on the above, I'll proceed to add more test cases in order to prevent regression. |
} | ||
|
||
// This test case is added to prevent regression. | ||
test("SPARK-22267 Spark SQL incorrectly reads ORC files when column order is different") { |
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.
This is added to prevent regression according to your request, @gatorsmile ~
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.
it's weird to have a test verifying a bug, I think it's good enough to have a JIRA tracking this bug.
Test build #82701 has finished for PR 19470 at commit
|
@@ -138,8 +138,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable | |||
if (maybePhysicalSchema.isEmpty) { |
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.
nit
val isEmptyFile = OrcFileOperator.readSchema(Seq(file.filePath), Some(conf)).isEmpty
if (isEmptyFile) {
...
} else ...
@@ -138,8 +138,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable | |||
if (maybePhysicalSchema.isEmpty) { | |||
Iterator.empty | |||
} else { | |||
val physicalSchema = maybePhysicalSchema.get | |||
OrcRelation.setRequiredColumns(conf, physicalSchema, requiredSchema) | |||
OrcRelation.setRequiredColumns(conf, dataSchema, requiredSchema) |
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.
does it work? seems here we lie to the orc reader about the physical schema.
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.
oh i see, we only need to pass the required column indices to orc reader.
case (field, ordinal) => | ||
var ref = oi.getStructFieldRef(field.name) | ||
if (ref == null) { | ||
val maybeIndex = dataSchema.getFieldIndex(field.name) |
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.
the requiredSchema
is guaranteed to be contained in the dataSchema
.
|
||
iterator.map { value => | ||
val raw = deserializer.deserialize(value) | ||
var i = 0 | ||
val length = fieldRefs.length | ||
while (i < length) { | ||
val fieldValue = oi.getStructFieldData(raw, fieldRefs(i)) | ||
val fieldRef = fieldRefs(i) | ||
val fieldValue = if (fieldRef == null) null else oi.getStructFieldData(raw, fieldRefs(i)) | ||
if (fieldValue == null) { |
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.
nit:
if (fieldRef == null) {
row.setNull...
} else {
val fieldValue = ...
...
}
) | ||
|
||
checkAnswer( | ||
sql(s"SELECT * FROM $db.t"), |
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.
please list all columns here instead of *
, to make the test more clear
Row(null, "12")) | ||
|
||
checkAnswer( | ||
sql(s"SELECT * FROM $db.t"), |
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.
ditto
LGTM except some minor comments |
@cloud-fan . Thank you so much for review! if (fieldRef == null) {
row.setNull...
} else {
val fieldValue = ...
...
} |
@@ -2050,4 +2050,60 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { | |||
} | |||
} | |||
} | |||
|
|||
test("SPARK-18355 Use Spark schema to read ORC table instead of ORC file schema") { |
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.
Improve the test case for checking the other formats?
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.
since it depends on the CONVERT_METASTORE_XXX conf, maybe also test parquet.
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.
Yep. I'll add parquet
, too.
LGTM, pending jenkins |
LGTM too. |
Seq("true", "false").foreach { value => | ||
withSQLConf( | ||
HiveUtils.CONVERT_METASTORE_ORC.key -> value, | ||
HiveUtils.CONVERT_METASTORE_PARQUET.key -> value) { |
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.
As you separate orc and parquet to two tests in fact, maybe you just need to test against one config at one time, i.e., orc -> HiveUtils.CONVERT_METASTORE_ORC, parquet -> HiveUtils.CONVERT_METASTORE_PARQUET.key.
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.
Thank you for review, @viirya . For that, yes, we can, but that will be a little-bit overkill.
One minor comment doesn't affect this. LGTM. |
Test build #82718 has finished for PR 19470 at commit
|
Test build #82720 has finished for PR 19470 at commit
|
Retest this please. |
R failure seems to be irrelevant.
|
Test build #82724 has finished for PR 19470 at commit
|
Now, it's passed again. :) |
… ORC table instead of ORC file schema Before Hive 2.0, ORC File schema has invalid column names like `_col1` and `_col2`. This is a well-known limitation and there are several Apache Spark issues with `spark.sql.hive.convertMetastoreOrc=true`. This PR ignores ORC File schema and use Spark schema. Pass the newly added test case. Author: Dongjoon Hyun <[email protected]> Closes #19470 from dongjoon-hyun/SPARK-18355. (cherry picked from commit e6e3600) Signed-off-by: Wenchen Fan <[email protected]>
thanks, merging to master/2.2! |
Thank you so much, @cloud-fan , @gatorsmile , and @viirya ! |
BTW, @cloud-fan . Could you review #18460 , too? I think we need your final approval. :) |
… ORC table instead of ORC file schema Before Hive 2.0, ORC File schema has invalid column names like `_col1` and `_col2`. This is a well-known limitation and there are several Apache Spark issues with `spark.sql.hive.convertMetastoreOrc=true`. This PR ignores ORC File schema and use Spark schema. Pass the newly added test case. Author: Dongjoon Hyun <[email protected]> Closes apache#19470 from dongjoon-hyun/SPARK-18355. (cherry picked from commit e6e3600) Signed-off-by: Wenchen Fan <[email protected]>
What changes were proposed in this pull request?
Before Hive 2.0, ORC File schema has invalid column names like
_col1
and_col2
. This is a well-known limitation and there are several Apache Spark issues withspark.sql.hive.convertMetastoreOrc=true
. This PR ignores ORC File schema and use Spark schema.How was this patch tested?
Pass the newly added test case.