[ https://issues.apache.org/jira/browse/SPARK-18681?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Apache Spark reassigned SPARK-18681: ------------------------------------ Assignee: (was: Apache Spark) > Throw Filtering is supported only on partition keys of type string exception > ---------------------------------------------------------------------------- > > Key: SPARK-18681 > URL: https://issues.apache.org/jira/browse/SPARK-18681 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 2.1.0 > Reporter: Yuming Wang > > I'm using MySQL for the Hive Metastore. > {{hive.metastore.try.direct.sql=true}} and > {{hive.metastore.integral.jdo.pushdown=false}} . It will throw following > exception when filtering on a *int* partition column after > [SPARK-17992|https://issues.apache.org/jira/browse/SPARK-17992]. > {noformat} > spark-sql> CREATE TABLE test (value INT) PARTITIONED BY (part INT); > Time taken: 0.221 seconds > spark-sql> select * from test where part=1 limit 10; > 16/12/02 08:33:45 ERROR thriftserver.SparkSQLDriver: Failed in [select * from > test where part=1 limit 10] > 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:610) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getPartitionsByFilter$1.apply(HiveClientImpl.scala:549) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getPartitionsByFilter$1.apply(HiveClientImpl.scala:547) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:282) > at > org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:229) > at > org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:228) > at > org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:271) > at > org.apache.spark.sql.hive.client.HiveClientImpl.getPartitionsByFilter(HiveClientImpl.scala:547) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$listPartitionsByFilter$1.apply(HiveExternalCatalog.scala:954) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$listPartitionsByFilter$1.apply(HiveExternalCatalog.scala:938) > at > org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:91) > at > org.apache.spark.sql.hive.HiveExternalCatalog.listPartitionsByFilter(HiveExternalCatalog.scala:938) > at > org.apache.spark.sql.hive.MetastoreRelation.getHiveQlPartitions(MetastoreRelation.scala:156) > at > org.apache.spark.sql.hive.execution.HiveTableScanExec$$anonfun$10.apply(HiveTableScanExec.scala:151) > at > org.apache.spark.sql.hive.execution.HiveTableScanExec$$anonfun$10.apply(HiveTableScanExec.scala:150) > at org.apache.spark.util.Utils$.withDummyCallSite(Utils.scala:2435) > at > org.apache.spark.sql.hive.execution.HiveTableScanExec.doExecute(HiveTableScanExec.scala:149) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113) > at > org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:225) > at > org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:308) > at > org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:38) > at > org.apache.spark.sql.execution.SparkPlan.executeCollectPublic(SparkPlan.scala:295) > at > org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$4.apply(QueryExecution.scala:134) > at > org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$4.apply(QueryExecution.scala:133) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57) > at > org.apache.spark.sql.execution.QueryExecution.hiveResultString(QueryExecution.scala:133) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:63) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:335) > at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:247) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at > org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:728) > at > org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:177) > at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:202) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:116) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > Caused by: java.lang.reflect.InvocationTargetException > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at > org.apache.spark.sql.hive.client.Shim_v0_13.getPartitionsByFilter(HiveShim.scala:597) > ... 44 more > Caused by: MetaException(message:Filtering is supported only on partition > keys of type string) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_by_filter_result$get_partitions_by_filter_resultStandardScheme.read(ThriftHiveMetastore.java) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_by_filter_result$get_partitions_by_filter_resultStandardScheme.read(ThriftHiveMetastore.java) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_by_filter_result.read(ThriftHiveMetastore.java) > at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:78) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_get_partitions_by_filter(ThriftHiveMetastore.java:2216) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_partitions_by_filter(ThriftHiveMetastore.java:2200) > at > org.apache.hadoop.hive.metastore.HiveMetaStoreClient.listPartitionsByFilter(HiveMetaStoreClient.java:1103) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at > org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:156) > at com.sun.proxy.$Proxy22.listPartitionsByFilter(Unknown Source) > at > org.apache.hadoop.hive.ql.metadata.Hive.getPartitionsByFilter(Hive.java:2254) > ... 49 more > 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:610) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getPartitionsByFilter$1.apply(HiveClientImpl.scala:549) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getPartitionsByFilter$1.apply(HiveClientImpl.scala:547) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:282) > at > org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:229) > at > org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:228) > at > org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:271) > at > org.apache.spark.sql.hive.client.HiveClientImpl.getPartitionsByFilter(HiveClientImpl.scala:547) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$listPartitionsByFilter$1.apply(HiveExternalCatalog.scala:954) > at > org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$listPartitionsByFilter$1.apply(HiveExternalCatalog.scala:938) > at > org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:91) > at > org.apache.spark.sql.hive.HiveExternalCatalog.listPartitionsByFilter(HiveExternalCatalog.scala:938) > at > org.apache.spark.sql.hive.MetastoreRelation.getHiveQlPartitions(MetastoreRelation.scala:156) > at > org.apache.spark.sql.hive.execution.HiveTableScanExec$$anonfun$10.apply(HiveTableScanExec.scala:151) > at > org.apache.spark.sql.hive.execution.HiveTableScanExec$$anonfun$10.apply(HiveTableScanExec.scala:150) > at org.apache.spark.util.Utils$.withDummyCallSite(Utils.scala:2435) > at > org.apache.spark.sql.hive.execution.HiveTableScanExec.doExecute(HiveTableScanExec.scala:149) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113) > at > org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:225) > at > org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:308) > at > org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:38) > at > org.apache.spark.sql.execution.SparkPlan.executeCollectPublic(SparkPlan.scala:295) > at > org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$4.apply(QueryExecution.scala:134) > at > org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$4.apply(QueryExecution.scala:133) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57) > at > org.apache.spark.sql.execution.QueryExecution.hiveResultString(QueryExecution.scala:133) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:63) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:335) > at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:247) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at > org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:728) > at > org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:177) > at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:202) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:116) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > Caused by: java.lang.reflect.InvocationTargetException > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at > org.apache.spark.sql.hive.client.Shim_v0_13.getPartitionsByFilter(HiveShim.scala:597) > ... 44 more > Caused by: MetaException(message:Filtering is supported only on partition > keys of type string) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_by_filter_result$get_partitions_by_filter_resultStandardScheme.read(ThriftHiveMetastore.java) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_by_filter_result$get_partitions_by_filter_resultStandardScheme.read(ThriftHiveMetastore.java) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_by_filter_result.read(ThriftHiveMetastore.java) > at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:78) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_get_partitions_by_filter(ThriftHiveMetastore.java:2216) > at > org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.get_partitions_by_filter(ThriftHiveMetastore.java:2200) > at > org.apache.hadoop.hive.metastore.HiveMetaStoreClient.listPartitionsByFilter(HiveMetaStoreClient.java:1103) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at > org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:156) > at com.sun.proxy.$Proxy22.listPartitionsByFilter(Unknown Source) > at > org.apache.hadoop.hive.ql.metadata.Hive.getPartitionsByFilter(Hive.java:2254) > ... 49 more > {noformat} > h4. Work around > # Modifying your Hive metastore configuration to set > {{hive.metastore.integral.jdo.pushdown=true}}, and *restart* Hive Metastore > Server. > # Modifying {{HiveShim.scala}} to reture all partitons like before, I think > this way is good to compatible with previously spark version. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org