You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "wyp (JIRA)" <ji...@apache.org> on 2019/08/16 09:20:00 UTC

[jira] [Comment Edited] (SPARK-28587) JDBC data source's partition whereClause should support jdbc dialect

    [ https://issues.apache.org/jira/browse/SPARK-28587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16908881#comment-16908881 ] 

wyp edited comment on SPARK-28587 at 8/16/19 9:19 AM:
------------------------------------------------------

[~maropu], Thank you for your reply. 

If the type of the TIMES field is timestamp, then 'SELECT 1 FROM search_info_test WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null' will fail. eg:
{code:java}
// CREATE TABLE SEARCH_INFO_TEST (ID BIGINT primary key, TIMES TIMESTAMP);
// CREATE INDEX TIMES_INDEX ON SEARCH_INFO_TEST(TIMES);

0: jdbc:phoenix:thin:url=http://192.168.0.1> SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null;
Error: Error -1 (00000) : Error while executing SQL "SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null": Remote driver error: RuntimeException: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00' -> TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00' (state=00000,code=-1)
org.apache.calcite.avatica.AvaticaSqlException: Error -1 (00000) : Error while executing SQL "SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null": Remote driver error: RuntimeException: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00' -> TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00'
at org.apache.calcite.avatica.Helper.createException(Helper.java:54)
at org.apache.calcite.avatica.Helper.createException(Helper.java:41)
at org.apache.calcite.avatica.AvaticaStatement.executeInternal(AvaticaStatement.java:163)
at org.apache.calcite.avatica.AvaticaStatement.execute(AvaticaStatement.java:217)
at sqlline.Commands.execute(Commands.java:822)
at sqlline.Commands.sql(Commands.java:732)
at sqlline.SqlLine.dispatch(SqlLine.java:813)
at sqlline.SqlLine.begin(SqlLine.java:686)
at sqlline.SqlLine.start(SqlLine.java:398)
at sqlline.SqlLine.main(SqlLine.java:291)
at org.apache.phoenix.queryserver.client.SqllineWrapper.main(SqllineWrapper.java:93)
java.lang.RuntimeException: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00'
at org.apache.calcite.avatica.jdbc.JdbcMeta.propagate(JdbcMeta.java:700)
at org.apache.calcite.avatica.jdbc.PhoenixJdbcMeta.prepareAndExecute(PhoenixJdbcMeta.java:101)
at org.apache.calcite.avatica.remote.LocalService.apply(LocalService.java:206)
at org.apache.calcite.avatica.remote.Service$PrepareAndExecuteRequest.accept(Service.java:927)
at org.apache.calcite.avatica.remote.Service$PrepareAndExecuteRequest.accept(Service.java:879)
at org.apache.calcite.avatica.remote.AbstractHandler.apply(AbstractHandler.java:94)
at org.apache.calcite.avatica.remote.ProtobufHandler.apply(ProtobufHandler.java:46)
at org.apache.calcite.avatica.server.AvaticaProtobufHandler.handle(AvaticaProtobufHandler.java:127)
at org.eclipse.jetty.server.handler.HandlerList.handle(HandlerList.java:52)
at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
at org.eclipse.jetty.server.Server.handle(Server.java:534)
at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:320)
at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108)
at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)
at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
at java.lang.Thread.run(Thread.java:834)
Caused by: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00'
at org.apache.phoenix.schema.TypeMismatchException.newException(TypeMismatchException.java:53)
at org.apache.phoenix.expression.ComparisonExpression.create(ComparisonExpression.java:149)
at org.apache.phoenix.compile.ExpressionCompiler.visitLeave(ExpressionCompiler.java:234)
at org.apache.phoenix.compile.ExpressionCompiler.visitLeave(ExpressionCompiler.java:146)
at org.apache.phoenix.parse.ComparisonParseNode.accept(ComparisonParseNode.java:47)
at org.apache.phoenix.parse.CompoundParseNode.acceptChildren(CompoundParseNode.java:64)
at org.apache.phoenix.parse.OrParseNode.accept(OrParseNode.java:46)
at org.apache.phoenix.compile.WhereCompiler.compile(WhereCompiler.java:145)
at org.apache.phoenix.compile.WhereCompiler.compile(WhereCompiler.java:124)
at org.apache.phoenix.compile.WhereCompiler.compile(WhereCompiler.java:109)
at org.apache.phoenix.compile.QueryCompiler.compileSingleFlatQuery(QueryCompiler.java:586)
at org.apache.phoenix.compile.QueryCompiler.compileSingleQuery(QueryCompiler.java:537)
at org.apache.phoenix.compile.QueryCompiler.compileSelect(QueryCompiler.java:206)
at org.apache.phoenix.compile.QueryCompiler.compile(QueryCompiler.java:166)
at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableSelectStatement.compilePlan(PhoenixStatement.java:417)
at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableSelectStatement.compilePlan(PhoenixStatement.java:383)
at org.apache.phoenix.jdbc.PhoenixStatement$1.call(PhoenixStatement.java:227)
at org.apache.phoenix.jdbc.PhoenixStatement$1.call(PhoenixStatement.java:216)
at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
at org.apache.phoenix.jdbc.PhoenixStatement.executeQuery(PhoenixStatement.java:215)
at org.apache.phoenix.jdbc.PhoenixStatement.executeQuery(PhoenixStatement.java:208)
at org.apache.phoenix.jdbc.PhoenixStatement.execute(PhoenixStatement.java:1844)
at org.apache.calcite.avatica.jdbc.PhoenixJdbcMeta.prepareAndExecute(PhoenixJdbcMeta.java:83)
... 20 more
{code}
If you write the sql like this 'SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < TO_TIMESTAMP('2019-07-31 04:00:00') or "TIMES" is null' , than it will work very well. 
{code:java}
0: jdbc:phoenix:thin:url=http://192.168.0.1> SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < TO_TIMESTAMP('2019-07-31 04:00:00') or "TIMES" is null;
+----+
| 1 |
+----+
+----+
No rows selected (0.061 seconds)
{code}
Oracle database also has this problem, o Oracle database has custom dialects (OracleDialect). I have fixed these issues in my private code repository ([https://github.com/397090770/spark/commit/c54c4a43b7e38ebfa250f8d268dabe82332b5407]), could you please help me review the change? Thank you.

 


was (Author: 397090770):
[~maropu], Thank you for your reply. 

If the type of the TIMES field is timestamp, then 'SELECT 1 FROM search_info_test WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null' will fail. eg:

 
{code:java}
// CREATE TABLE SEARCH_INFO_TEST (ID BIGINT primary key, TIMES TIMESTAMP);
// CREATE INDEX TIMES_INDEX ON SEARCH_INFO_TEST(TIMES);

0: jdbc:phoenix:thin:url=http://192.168.0.1> SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null;
Error: Error -1 (00000) : Error while executing SQL "SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null": Remote driver error: RuntimeException: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00' -> TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00' (state=00000,code=-1)
org.apache.calcite.avatica.AvaticaSqlException: Error -1 (00000) : Error while executing SQL "SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null": Remote driver error: RuntimeException: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00' -> TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00'
at org.apache.calcite.avatica.Helper.createException(Helper.java:54)
at org.apache.calcite.avatica.Helper.createException(Helper.java:41)
at org.apache.calcite.avatica.AvaticaStatement.executeInternal(AvaticaStatement.java:163)
at org.apache.calcite.avatica.AvaticaStatement.execute(AvaticaStatement.java:217)
at sqlline.Commands.execute(Commands.java:822)
at sqlline.Commands.sql(Commands.java:732)
at sqlline.SqlLine.dispatch(SqlLine.java:813)
at sqlline.SqlLine.begin(SqlLine.java:686)
at sqlline.SqlLine.start(SqlLine.java:398)
at sqlline.SqlLine.main(SqlLine.java:291)
at org.apache.phoenix.queryserver.client.SqllineWrapper.main(SqllineWrapper.java:93)
java.lang.RuntimeException: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00'
at org.apache.calcite.avatica.jdbc.JdbcMeta.propagate(JdbcMeta.java:700)
at org.apache.calcite.avatica.jdbc.PhoenixJdbcMeta.prepareAndExecute(PhoenixJdbcMeta.java:101)
at org.apache.calcite.avatica.remote.LocalService.apply(LocalService.java:206)
at org.apache.calcite.avatica.remote.Service$PrepareAndExecuteRequest.accept(Service.java:927)
at org.apache.calcite.avatica.remote.Service$PrepareAndExecuteRequest.accept(Service.java:879)
at org.apache.calcite.avatica.remote.AbstractHandler.apply(AbstractHandler.java:94)
at org.apache.calcite.avatica.remote.ProtobufHandler.apply(ProtobufHandler.java:46)
at org.apache.calcite.avatica.server.AvaticaProtobufHandler.handle(AvaticaProtobufHandler.java:127)
at org.eclipse.jetty.server.handler.HandlerList.handle(HandlerList.java:52)
at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
at org.eclipse.jetty.server.Server.handle(Server.java:534)
at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:320)
at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108)
at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)
at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
at java.lang.Thread.run(Thread.java:834)
Caused by: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00'
at org.apache.phoenix.schema.TypeMismatchException.newException(TypeMismatchException.java:53)
at org.apache.phoenix.expression.ComparisonExpression.create(ComparisonExpression.java:149)
at org.apache.phoenix.compile.ExpressionCompiler.visitLeave(ExpressionCompiler.java:234)
at org.apache.phoenix.compile.ExpressionCompiler.visitLeave(ExpressionCompiler.java:146)
at org.apache.phoenix.parse.ComparisonParseNode.accept(ComparisonParseNode.java:47)
at org.apache.phoenix.parse.CompoundParseNode.acceptChildren(CompoundParseNode.java:64)
at org.apache.phoenix.parse.OrParseNode.accept(OrParseNode.java:46)
at org.apache.phoenix.compile.WhereCompiler.compile(WhereCompiler.java:145)
at org.apache.phoenix.compile.WhereCompiler.compile(WhereCompiler.java:124)
at org.apache.phoenix.compile.WhereCompiler.compile(WhereCompiler.java:109)
at org.apache.phoenix.compile.QueryCompiler.compileSingleFlatQuery(QueryCompiler.java:586)
at org.apache.phoenix.compile.QueryCompiler.compileSingleQuery(QueryCompiler.java:537)
at org.apache.phoenix.compile.QueryCompiler.compileSelect(QueryCompiler.java:206)
at org.apache.phoenix.compile.QueryCompiler.compile(QueryCompiler.java:166)
at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableSelectStatement.compilePlan(PhoenixStatement.java:417)
at org.apache.phoenix.jdbc.PhoenixStatement$ExecutableSelectStatement.compilePlan(PhoenixStatement.java:383)
at org.apache.phoenix.jdbc.PhoenixStatement$1.call(PhoenixStatement.java:227)
at org.apache.phoenix.jdbc.PhoenixStatement$1.call(PhoenixStatement.java:216)
at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
at org.apache.phoenix.jdbc.PhoenixStatement.executeQuery(PhoenixStatement.java:215)
at org.apache.phoenix.jdbc.PhoenixStatement.executeQuery(PhoenixStatement.java:208)
at org.apache.phoenix.jdbc.PhoenixStatement.execute(PhoenixStatement.java:1844)
at org.apache.calcite.avatica.jdbc.PhoenixJdbcMeta.prepareAndExecute(PhoenixJdbcMeta.java:83)
... 20 more
{code}
If you write the sql like this 'SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < TO_TIMESTAMP('2019-07-31 04:00:00') or "TIMES" is null' , than it will work very well.

 

 
{code:java}
0: jdbc:phoenix:thin:url=http://192.168.0.1> SELECT 1 FROM SEARCH_INFO_TEST WHERE "TIMES" < TO_TIMESTAMP('2019-07-31 04:00:00') or "TIMES" is null;
+----+
| 1 |
+----+
+----+
No rows selected (0.061 seconds)
{code}
This problem exists in the same way as an Oracle database, So Oracle database has custom dialects (OracleDialect). I have fixed these issues in my private code repository ([https://github.com/397090770/spark/commit/c54c4a43b7e38ebfa250f8d268dabe82332b5407]), could you please help me review the change? Thank you.

 

> JDBC data source's partition whereClause should support jdbc dialect
> --------------------------------------------------------------------
>
>                 Key: SPARK-28587
>                 URL: https://issues.apache.org/jira/browse/SPARK-28587
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 2.4.3
>            Reporter: wyp
>            Priority: Minor
>
> When we use JDBC data source to search data from Phoenix, and use timestamp data type column for partitionColumn, e.g.
> {code:java}
> val url = "jdbc:phoenix:thin:url=localhost:8765;serialization=PROTOBUF"
> val driver = "org.apache.phoenix.queryserver.client.Driver"
> val df = spark.read.format("jdbc")
>     .option("url", url)
>     .option("driver", driver)
>     .option("fetchsize", "1000")
>     .option("numPartitions", "6")
>     .option("partitionColumn", "times")
>     .option("lowerBound", "2019-07-31 00:00:00")
>     .option("upperBound", "2019-08-01 00:00:00")
>     .option("dbtable", "search_info_test")
>     .load().select("id")
> println(df.count())
> {code}
> there will throw AvaticaSqlException in phoenix:
> {code:java}
> org.apache.calcite.avatica.AvaticaSqlException: Error -1 (00000) : while preparing SQL: SELECT 1 FROM search_info_test WHERE "TIMES" < '2019-07-31 04:00:00' or "TIMES" is null
>   at org.apache.calcite.avatica.Helper.createException(Helper.java:54)
>   at org.apache.calcite.avatica.Helper.createException(Helper.java:41)
>   at org.apache.calcite.avatica.AvaticaConnection.prepareStatement(AvaticaConnection.java:368)
>   at org.apache.calcite.avatica.AvaticaConnection.prepareStatement(AvaticaConnection.java:299)
>   at org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD.compute(JDBCRDD.scala:300)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
>   at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
>   at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
>   at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
>   at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:55)
>   at org.apache.spark.scheduler.Task.run(Task.scala:121)
>   at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
>   at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
>   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>   at java.lang.Thread.run(Thread.java:748)
> java.lang.RuntimeException: org.apache.phoenix.schema.TypeMismatchException: ERROR 203 (22005): Type mismatch. TIMESTAMP and VARCHAR for "TIMES" < '2019-07-31 04:00:00'
>   at org.apache.calcite.avatica.jdbc.JdbcMeta.propagate(JdbcMeta.java:700)
>   at org.apache.calcite.avatica.jdbc.PhoenixJdbcMeta.prepare(PhoenixJdbcMeta.java:67)
>   at org.apache.calcite.avatica.remote.LocalService.apply(LocalService.java:195)
>   at org.apache.calcite.avatica.remote.Service$PrepareRequest.accept(Service.java:1215)
>   at org.apache.calcite.avatica.remote.Service$PrepareRequest.accept(Service.java:1186)
>   at org.apache.calcite.avatica.remote.AbstractHandler.apply(AbstractHandler.java:94)
>   at org.apache.calcite.avatica.remote.ProtobufHandler.apply(ProtobufHandler.java:46)
>   at org.apache.calcite.avatica.server.AvaticaProtobufHandler.handle(AvaticaProtobufHandler.java:127)
>   at org.eclipse.jetty.server.handler.HandlerList.handle(HandlerList.java:52)
>   at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
>   at org.eclipse.jetty.server.Server.handle(Server.java:534)
>   at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:320)
>   at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
>   at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
>   at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108)
>   at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
>   at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
>   at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
>   at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)
>   at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
>   at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
>   at java.lang.Thread.run(Thread.java:834)
> {code}
> the reason is because JDBC data source's partition whereClause doesn't support jdbc dialect. We should use jdbc dialect to compile '2019-07-31 04:00:00' to to_timestamp('2019-07-31 04:00:00')



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

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