[ https://issues.apache.org/jira/browse/SPARK-21063?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16196354#comment-16196354 ]
Andreas Weise commented on SPARK-21063: --------------------------------------- Seems like a bug IMHO. Same problem here with Spark 2.2.0, Hive on MySQL and default HiveThriftServer2. {code:title=Creating Table} spark.range(10).write.mode('overwrite').format("parquet").saveAsTable("test1") {code} {code:title=beeline is fine} $ bin/beeline Beeline version 1.2.1.spark2 by Apache Hive beeline> !connect jdbc:hive2://localhost:10000 Connecting to jdbc:hive2://localhost:10000 Connected to: Spark SQL (version 2.2.0) Driver: Hive JDBC (version 1.2.1.spark2) Transaction isolation: TRANSACTION_REPEATABLE_READ 0: jdbc:hive2://localhost:10000> select * from test1; +-----+--+ | id | +-----+--+ | 5 | | 6 | | 7 | | 8 | | 9 | | 0 | | 1 | | 2 | | 3 | | 4 | +-----+--+ 10 rows selected (0.213 seconds) {code} {code:title=spark+jdbc gives 0 results} spark.read.jdbc("jdbc:hive2://localhost:10000/default","test1", properties={"driver": "org.apache.hive.jdbc.HiveDriver"}).collect() [] {code} Whats interesting now is the SQL Statements fired during JDBC call from Spark in contrast to the one from beeline. They are all logged in HiveThriftServer2: {code:title=SQL from Spark+JDBC} SELECT * FROM test1 WHERE 1=0 SELECT "id" FROM test1 {code} {code:title=SQL from beeline} select * from test1 {code} Now tried Spark+JDBC with fetchsize option: {code:title=Spark+JDBC with fetchsize} spark.read.jdbc("jdbc:hive2://localhost:10000/default","test1", properties={"driver": "org.apache.hive.jdbc.HiveDriver", "fetchsize": "10"}).collect() {code} This results in an error. {noformat} org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 28.0 failed 4 times, most recent failure: Lost task 0.3 in stage 28.0 (TID 52, 192.168.0.210, executor 10): java.sql.SQLException: Cannot convert column 1 to long: java.lang.NumberFormatException: For input string: "id" at org.apache.hive.jdbc.HiveBaseResultSet.getLong(HiveBaseResultSet.java:372) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$org$apache$spark$sql$execution$datasources$jdbc$JdbcUtils$$makeGetter$8.apply(JdbcUtils.scala:409) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$org$apache$spark$sql$execution$datasources$jdbc$JdbcUtils$$makeGetter$8.apply(JdbcUtils.scala:408) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anon$1.getNext(JdbcUtils.scala:330) at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anon$1.getNext(JdbcUtils.scala:312) at org.apache.spark.util.NextIterator.hasNext(NextIterator.scala:73) at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37) at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:32) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:395) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:234) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:228) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:335) 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) Caused by: java.lang.NumberFormatException: For input string: "id" at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) at java.lang.Long.parseLong(Long.java:589) at java.lang.Long.valueOf(Long.java:803) at org.apache.hive.jdbc.HiveBaseResultSet.getLong(HiveBaseResultSet.java:368) ... 23 more {noformat} > Spark return an empty result from remote hadoop cluster > ------------------------------------------------------- > > Key: SPARK-21063 > URL: https://issues.apache.org/jira/browse/SPARK-21063 > Project: Spark > Issue Type: Bug > Components: Spark Core, SQL > Affects Versions: 2.1.0, 2.1.1 > Reporter: Peter Bykov > > Spark returning empty result from when querying remote hadoop cluster. > All firewall settings removed. > Querying using JDBC working properly using hive-jdbc driver from version 1.1.1 > Code snippet is: > {code:java} > val spark = SparkSession.builder > .appName("RemoteSparkTest") > .master("local") > .getOrCreate() > val df = spark.read > .option("url", "jdbc:hive2://remote.hive.local:10000/default") > .option("user", "user") > .option("password", "pass") > .option("dbtable", "test_table") > .option("driver", "org.apache.hive.jdbc.HiveDriver") > .format("jdbc") > .load() > > df.show() > {code} > Result: > {noformat} > +-------------------+ > |test_table.test_col| > +-------------------+ > +-------------------+ > {noformat} > All manipulations like: > {code:java} > df.select(*).show() > {code} > returns empty result too. -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org