You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2020/11/19 13:17:27 UTC

[spark] branch branch-2.4 updated: [SPARK-27421][SQL][2.4] Fix filter for int column and value class java.lang.String when pruning partition column

This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new 3772bfa  [SPARK-27421][SQL][2.4] Fix filter for int column and value class java.lang.String when pruning partition column
3772bfa is described below

commit 3772bfa6ddc3360377f60dc37aaa7cd031cb5bac
Author: Yuming Wang <yu...@ebay.com>
AuthorDate: Thu Nov 19 22:14:11 2020 +0900

    [SPARK-27421][SQL][2.4] Fix filter for int column and value class java.lang.String when pruning partition column
    
    This pr backport https://github.com/apache/spark/pull/30380 to branch-2.4.
    
    ### What changes were proposed in this pull request?
    
    This pr fix filter for int column and value class java.lang.String when pruning partition column.
    
    How to reproduce this issue:
    ```scala
    spark.sql("CREATE table test (name STRING) partitioned by (id int) STORED AS PARQUET")
    spark.sql("CREATE VIEW test_view as select cast(id as string) as id, name from test")
    spark.sql("SELECT * FROM test_view WHERE id = '0'").explain
    ```
    ```
    20/11/15 06:19:01 INFO audit: ugi=root ip=unknown-ip-addr cmd=get_partitions_by_filter : db=default tbl=test
    20/11/15 06:19:01 INFO MetaStoreDirectSql: Unable to push down SQL filter: Cannot push down filter for int column and value class java.lang.String
    20/11/15 06:19:01 ERROR SparkSQLDriver: Failed in [SELECT * FROM test_view WHERE id = '0']
    java.lang.RuntimeException: Caught Hive MetaException attempting to get partition metadata by filter from Hive. You can set the Spark configuration setting spark.sql.hive.manageFilesourcePartitions to false to work around this problem, however this will result in degraded performance. Please report a bug: https://issues.apache.org/jira/browse/SPARK
     at org.apache.spark.sql.hive.client.Shim_v0_13.getPartitionsByFilter(HiveShim.scala:828)
     at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getPartitionsByFilter$1(HiveClientImpl.scala:745)
     at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:294)
     at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:227)
     at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:226)
     at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:276)
     at org.apache.spark.sql.hive.client.HiveClientImpl.getPartitionsByFilter(HiveClientImpl.scala:743)
    ```
    
    ### Why are the changes needed?
    
    Fix bug.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    Unit test.
    
    Closes #30422 from wangyum/SPARK-27421-2.4.
    
    Authored-by: Yuming Wang <yu...@ebay.com>
    Signed-off-by: HyukjinKwon <gu...@apache.org>
---
 .../main/scala/org/apache/spark/sql/hive/client/HiveShim.scala   | 2 +-
 .../scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala | 9 ++++++++-
 2 files changed, 9 insertions(+), 2 deletions(-)

diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
index 095b0aa..cee795d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala
@@ -679,7 +679,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 {
       def unapply(expr: Expression): Option[Attribute] = {
         expr match {
           case attr: Attribute => Some(attr)
-          case Cast(child @ AtomicType(), dt: AtomicType, _)
+          case Cast(child @ IntegralType(), dt: IntegralType, _)
               if Cast.canSafeCast(child.dataType.asInstanceOf[AtomicType], dt) => unapply(child)
           case _ => None
         }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
index 5bdb13a..2568de4 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
@@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.apache.spark.sql.catalyst.catalog._
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType}
+import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType, StringType}
 import org.apache.spark.util.Utils
 
 // TODO: Refactor this to `HivePartitionFilteringSuite`
@@ -258,6 +258,13 @@ class HiveClientSuite(version: String)
     buildClient(new Configuration(), sharesHadoopClasses = false)
   }
 
+  test("getPartitionsByFilter: chunk in ('ab', 'ba') and ((cast(ds as string)>'20170102')") {
+    val day = (20170101 to 20170103, 0 to 23, Seq("ab", "ba"))
+    testMetastorePartitionFiltering(
+      attr("chunk").in("ab", "ba") && (attr("ds").cast(StringType) > "20170102"),
+      day :: Nil)
+  }
+
   private def testMetastorePartitionFiltering(
       filterExpr: Expression,
       expectedDs: Seq[Int],


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org