-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-19575][SQL]Reading from or writing to a hive serde table with a non pre-existing location should succeed #16910
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-19575][SQL]Reading from or writing to a hive serde table with a non pre-existing location should succeed #16910
Conversation
…a non pre-existing location should succeed
|
Test build #72808 has started for PR 16910 at commit |
|
retest this please |
|
Test build #72812 has finished for PR 16910 at commit
|
|
Test build #72815 has finished for PR 16910 at commit
|
|
@gatorsmile could you help to review this? thanks :) |
| dir.delete() | ||
| checkAnswer(spark.table("t"), Nil) | ||
|
|
||
| val newDir = dir.getAbsolutePath.stripSuffix("/") + "/x" |
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.
-> new File(dir, "x")
| } | ||
| } | ||
|
|
||
| test("read data from a hive serde table which has a not existed location should succeed") { |
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 the only test case failed without this fix. 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.
Yes, it is~
| s""" | ||
| |CREATE TABLE t(a string, b int) | ||
| |USING hive | ||
| |OPTIONS(path "file:${dir.getAbsolutePath}") |
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.
Indent issues.
|
|
||
| // if the table location is not exists, return an empty RDD | ||
| if (!fs.exists(locationPath)) { | ||
| return new EmptyRDD[InternalRow](sparkSession.sparkContext) |
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.
Can we do it in makeRDDForTable?
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 do it here for both non-partition table and partition table,while the partition table run well when the location does not exist with verifyPartitionPath set true,If we also want to run well when verifyPartitionPath set false, we should also do it in makeRDDForPartitionTable, then under this situation I do it here for both non-partition table and partition table.
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 seems not reasonable to do this for partition table, because the real partition path maybe not under the location of the partition table, I moved this logic to makeRDDForTable
|
Test build #73191 has finished for PR 16910 at commit
|
|
retest this please |
| val locationPath = new Path(inputPathStr) | ||
| val fs = locationPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) | ||
|
|
||
| // if the table location is not exists, return an empty RDD |
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.
is not exists -> does not exist
|
Test build #73193 has finished for PR 16910 at commit
|
|
Test build #73194 has finished for PR 16910 at commit
|
|
Test build #73196 has finished for PR 16910 at commit
|
|
@gatorsmile I have fixed the review above~ |
| HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) | ||
| } | ||
| val locationPath = new Path(inputPathStr) | ||
| val fs = locationPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) |
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.
How about replacing sparkSession.sessionState.newHadoopConf() by broadcastedHadoopConf.value.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.
ok~
| |PARTITIONED BY(a, b) | ||
| |LOCATION "file:${dir.getCanonicalPath}" | ||
| """.stripMargin) | ||
| val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("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.
useless?
|
|
||
| val newDirFile = new File(dir, "x") | ||
| spark.sql(s"ALTER TABLE t PARTITION(a=1, b=2) SET LOCATION " + | ||
| s"'${newDirFile.getAbsolutePath}'") |
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.
shorten it to a single line?
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.
101 characters...
let me modify some code.
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.
Actually, 101 is still ok
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.
e...is't it 100? let me test it...
I have modify some code to make it moer clear
|
|
||
| val newDirFile = new File(dir, "x") | ||
| spark.sql(s"ALTER TABLE t PARTITION(a=1, b=2) SET LOCATION " + | ||
| s"'${newDirFile.getAbsolutePath}'") |
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.
shorten it to a single line?
|
LGTM except a few minor comments. |
|
Test build #73266 has started for PR 16910 at commit |
|
retest this please |
|
retest this please |
|
retest this please |
|
Test build #73657 has finished for PR 16910 at commit
|
|
Test build #73661 has finished for PR 16910 at commit
|
|
Test build #73663 has finished for PR 16910 at commit
|
|
Test build #73666 has finished for PR 16910 at commit
|
| val fs = dirPath.getFileSystem(spark.sessionState.newHadoopConf()) | ||
| assert(new Path(table.location) == fs.makeQualified(dirPath)) | ||
|
|
||
| val tableLocFile = new File(table.location.stripPrefix("file:")) |
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.
new File(new URI(table.location))? please avoid .stripPrefix("file:") which looks very hacky.
| checkAnswer(spark.table("t"), Row("c", 1) :: Nil) | ||
|
|
||
| val newDirFile = new File(dir, "x") | ||
| val newDirPath = newDirFile.getAbsolutePath.stripSuffix("/") |
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.
.stripSuffix("/") is it needed?
|
Test build #73677 has finished for PR 16910 at commit
|
|
can you resolve the conflict? |
|
ok, do it now ~ yesterday is ok... |
|
Test build #73735 has started for PR 16910 at commit |
| s""" | ||
| |CREATE TABLE t(a string, b int) | ||
| |USING hive | ||
| |LOCATION '$dir' |
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.
can we just call dir.delete before creating this table?
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~
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.
@cloud-fan I found the dir will be created in create table, so we should keep current logic.
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 hive have the same behavior?
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, I test it in Hive
create table test(a string) location 'hdfs:/xx';
then hdfs:/xx will be created
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.
seems the InMemoryCatalog doesn't do this, you can send a new PR to fix it.
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 thanks~
|
retest this please |
|
Test build #73747 has finished for PR 16910 at commit
|
|
Test build #73829 has started for PR 16910 at commit |
|
retest this please |
|
Test build #73831 has finished for PR 16910 at commit
|
| spark.sql(s"ALTER TABLE t PARTITION(a=1, b=2) SET LOCATION '$newDirPath'") | ||
| assert(!newDirFile.exists()) | ||
| // select from a partition which location has changed to a not existed location | ||
| withSQLConf(SQLConf.HIVE_VERIFY_PARTITION_PATH.key -> "true") { |
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.
why setting this conf?
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.
if we don't set it,it will throw an exception,if we set it,it will check if the partition path exists,and will not throw exception just return emptyrdd even if path not existed
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.
is this expected? I think hive will always return empty result 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.
BTW this conf will be removed soon, as it has bugs.
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~thanks~ then here we also need to modify something?
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, hive return empty , if there is a bug here(could you describe what the bug is?), we can remove the conf ,and always return result?
|
Should we just close it now? |
|
Test build #91357 has finished for PR 16910 at commit
|
What changes were proposed in this pull request?
This PR is a folllowup work from SPARK-19329 - PR(#16672), which has unify the action when we reading from or writing to a datasource table with a non pre-existing locaiton, so here we should also unify the hive serde tables.
That is :
Currently when we select from a hive serde table which has a non pre-existing location will throw an exception:
How was this patch tested?
unit tests added