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

[spark] branch branch-3.0 updated: [SPARK-27421][SQL] 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.

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


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

commit 1101938d6189f5d4ecf1ffc23eee841e038722d7
Author: Yuming Wang <yu...@ebay.com>
AuthorDate: Thu Nov 19 14:01:42 2020 +0800

    [SPARK-27421][SQL] Fix filter for int column and value class java.lang.String when pruning partition column
    
    ### 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 #30380 from wangyum/SPARK-27421.
    
    Authored-by: Yuming Wang <yu...@ebay.com>
    Signed-off-by: Yuming Wang <yu...@ebay.com>
    (cherry picked from commit 014e1fbb3aba81a803c963fc0b7f4a8d1d70e253)
    Signed-off-by: Yuming Wang <yu...@ebay.com>
---
 .../main/scala/org/apache/spark/sql/hive/client/HiveShim.scala   | 2 +-
 .../spark/sql/hive/client/HivePartitionFilteringSuite.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 c7cbf32..d11bf94 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
@@ -729,7 +729,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.canUpCast(child.dataType.asInstanceOf[AtomicType], dt) => unapply(child)
           case _ => None
         }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala
index 2d615f6..edb1a27 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 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, StructType}
+import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType, StringType, StructType}
 import org.apache.spark.util.Utils
 
 class HivePartitionFilteringSuite(version: String)
@@ -276,6 +276,13 @@ class HivePartitionFilteringSuite(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 4, 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