[ https://issues.apache.org/jira/browse/SPARK-28587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
wyp updated SPARK-28587: ------------------------ Description: 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", "search_info_test") .option("lowerBound", "2019-07-31 00:00:00") .option("upperBound", "2019-08-01 00:00:00") .option("dbtable", "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') was: 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", "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') > 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", "search_info_test") > .option("lowerBound", "2019-07-31 00:00:00") > .option("upperBound", "2019-08-01 00:00:00") > .option("dbtable", "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-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org