-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-6910] [SQL] Support for pushing predicates down to metastore for partition pruning #6921
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
Conversation
|
ok to test |
|
Test build #35427 has finished for PR 6921 at commit
|
|
Test build #35431 has finished for PR 6921 at commit
|
I don't understand this. When I run the tests on my laptop, they all pass. Here is the command that I use- Any idea? |
|
test this please |
|
Test build #35493 has finished for PR 6921 at commit
|
|
I am rebasing it now. |
|
@piaozhexiu, thanks for working on this! I did some more investigation on what types of filters seem to be supported. It seems like we might be able to support scala> sql("CREATE TABLE test (key INT) PARTITIONED BY (i INT, s STRING)")
// works
scala> org.apache.spark.sql.hive.test.TestHive.catalog.client.getPartitionsByFilter(testTable, "s='11'")
// works
scala> org.apache.spark.sql.hive.test.TestHive.catalog.client.getPartitionsByFilter(testTable, "s>'11'")
// works
scala> org.apache.spark.sql.hive.test.TestHive.catalog.client.getPartitionsByFilter(testTable, "i=11")
// works
scala> org.apache.spark.sql.hive.test.TestHive.catalog.client.getPartitionsByFilter(testTable, "i>11")
// fails: Filtering is supported only on partition keys of type string:
scala> org.apache.spark.sql.hive.test.TestHive.catalog.client.getPartitionsByFilter(testTable, "i>'11'")I'm curious what problems you ran into exactly and which version of hive were you using when you encountered problems? A few other comments on the structure of this change?
What do you think? |
|
@marmbrus thanks for your comments! Your suggestions make sense. Let me do that. As for Hive filter types, the problems that I ran into are Spark unit test failures (hive/test). Since Spark builds against Hive 0.13, I've been looking at its source code to understand issues. But perhaps I was using incorrect encoding like you said. Let me verify that. |
|
Test build #35495 has finished for PR 6921 at commit
|
|
Test build #35891 has finished for PR 6921 at commit
|
|
Test build #35893 has finished for PR 6921 at commit
|
|
Test build #35915 has finished for PR 6921 at commit
|
917c3ee to
c0b82be
Compare
|
Test build #35924 has finished for PR 6921 at commit
|
|
@marmbrus I got all the unit tests passing now. Can you please review my patch? This is no longer WIP. I incorporated all your suggestions. A couple of things that I should mention-
Thanks! |
|
@marmbrus While digging into Hive commit log, I found this HIVE-4888. Binary comparison operators have been supported by So my patch will break backward compatibility with Hive 0.12 and older. I think the best we can do is to disable predicate pushdown in metastore if |
|
Test build #35947 has finished for PR 6921 at commit
|
|
I added a check for Hive metastore version, and if it is lower than 0.13, predicates are not pushed down. I think I handled all the cases now. Ready for review. |
|
One last update (hopefully). I realized that I can push down predicates even when Here is the summary of the current status:
|
|
Test build #36012 has finished for PR 6921 at commit
|
|
Test build #36026 has finished for PR 6921 at commit
|
|
This is weird. A couple of different tests randomly fail at each run. I am wondering whether this is because derby db intermittently fails during unit tests. I'll force another run by rebasing the patch. |
|
Test build #36044 has finished for PR 6921 at commit
|
|
Closing as it is superseded by #7216. |
…or partition pruning This PR supersedes my old one apache#6921. Since my patch has changed quite a bit, I am opening a new PR to make it easier to review. The changes include- * Implement `toMetastoreFilter()` function in `HiveShim` that takes `Seq[Expression]` and converts them into a filter string for Hive metastore. * This functions matches all the `AttributeReference` + `BinaryComparisonOp` + `Integral/StringType` patterns in `Seq[Expression]` and fold them into a string. * Change `hiveQlPartitions` field in `MetastoreRelation` to `getHiveQlPartitions()` function that takes a filter string parameter. * Call `getHiveQlPartitions()` in `HiveTableScan` with a filter string. But there are some cases in which predicate pushdown is disabled- Case | Predicate pushdown ------- | ----------------------------- Hive integral and string types | Yes Hive varchar type | No Hive 0.13 and newer | Yes Hive 0.12 and older | No convertMetastoreParquet=false | Yes convertMetastoreParquet=true | No In case of `convertMetastoreParquet=true`, predicates are not pushed down because this conversion happens in an `Analyzer` rule (`HiveMetastoreCatalog.ParquetConversions`). At this point, `HiveTableScan` hasn't run, so predicates are not available. But reading the source code, I think it is intentional to convert the entire Hive table w/ all the partitions into `ParquetRelation` because then `ParquetRelation` can be cached and reused for any query against that table. Please correct me if I am wrong. cc marmbrus Author: Cheolsoo Park <[email protected]> Closes apache#7216 from piaozhexiu/SPARK-6910-2 and squashes the following commits: aa1490f [Cheolsoo Park] Fix ordering of imports c212c4d [Cheolsoo Park] Incorporate review comments 5e93f9d [Cheolsoo Park] Predicate pushdown into Hive metastore
@marmbrus per our email conversation, I am sending a PR that implements the idea that I outlined in the jira FYI.
Analyzercaptures predicates if aunresolved relationis followed by afilter expressionand pushes them down intoHiveMetastoreCatalog.HiveMetastoreCatalogextracts partition predicates whose types are string and integral and constructs a string representation for them.getPartitionsByFilter(Table tbl, String filter)with the filter string.In addition, I understand better the limitations of Hive
getPartitionsByFilter(Table tbl, String filter)function now.The following is where the "filtering is supported only on partition keys of type string" error from in Hive-
As can be seen, one of following conditions has to be met-
doesOperatorSupportIntegral()returns true.)But looking at the
doesOperatorSupportIntegral()function, only equals and not-equals operators support integral types-This is not good because
<,>,<=, and>=which are very common in practice are not supported.In this PR, I restricted predicate pushdown to
=. With that restriction, all the unit tests pass now.