[
https://issues.apache.org/jira/browse/SPARK-5498?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16401701#comment-16401701
]
Lijia Liu edited comment on SPARK-5498 at 3/16/18 10:10 AM:
------------------------------------------------------------
In our cluster, we use hive 1.2, spark 2.2, hadoop 2.7. When we read hive table
use spark, we will get the error below:
{code:java}
ERROR SparkSQLDriver: Failed in [select * from test_par1 where b='2']
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in
stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID
3, localhost, executor driver): java.lang.ClassCastException:
org.apache.hadoop.io.Text cannot be cast to org.apache.hadoop.io.LongWritable
at
org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableLongObjectInspector.get(WritableLongObjectInspector.java:36)
at
org.apache.spark.sql.hive.HadoopTableReader$$anonfun$13$$anonfun$apply$6.apply(TableReader.scala:398)
at
org.apache.spark.sql.hive.HadoopTableReader$$anonfun$13$$anonfun$apply$6.apply(TableReader.scala:398)
at
org.apache.spark.sql.hive.HadoopTableReader$$anonfun$fillObject$2.apply(TableReader.scala:439)
at
org.apache.spark.sql.hive.HadoopTableReader$$anonfun$fillObject$2.apply(TableReader.scala:430)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
at
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:235)
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:1142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
{code}
repreduce this issue:
{code:java}
create table test_par(a string)
PARTITIONED BY (`b` bigint)
ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
STORED AS
INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat'
OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat';
ALTER TABLE test_par CHANGE a a bigint restrict; -- in hive
select * from test_par;
{code}
was (Author: liutang123):
In our cluster, we use hive 1.2, spark 2.2, hadoop 2.7. When we read hive table
use spark, we will get the error below:
{code:java}
ERROR SparkSQLDriver: Failed in [select * from test_par1 where b='2']
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in
stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID
3, localhost, executor driver): java.lang.ClassCastException:
org.apache.hadoop.io.Text cannot be cast to org.apache.hadoop.io.LongWritable
at
org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableLongObjectInspector.get(WritableLongObjectInspector.java:36)
at
org.apache.spark.sql.hive.HadoopTableReader$$anonfun$13$$anonfun$apply$6.apply(TableReader.scala:398)
at
org.apache.spark.sql.hive.HadoopTableReader$$anonfun$13$$anonfun$apply$6.apply(TableReader.scala:398)
at
org.apache.spark.sql.hive.HadoopTableReader$$anonfun$fillObject$2.apply(TableReader.scala:439)
at
org.apache.spark.sql.hive.HadoopTableReader$$anonfun$fillObject$2.apply(TableReader.scala:430)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
at
org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:235)
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:1142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
{code}
repreduce this issue:
{code:java}
create table test_par(a string)
PARTITIONED BY (`b` bigint)
ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
STORED AS
INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat'
OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat';
ALTER TABLE test_par CHANGE a a bigint restrict;
select * from test_par;
{code}
> [SPARK-SQL]when the partition schema does not match table schema,it throws
> java.lang.ClassCastException and so on
> -----------------------------------------------------------------------------------------------------------------
>
> Key: SPARK-5498
> URL: https://issues.apache.org/jira/browse/SPARK-5498
> Project: Spark
> Issue Type: Bug
> Components: SQL
> Affects Versions: 1.2.0, 2.2.0
> Reporter: jeanlyn
> Assignee: jeanlyn
> Priority: Major
> Fix For: 1.4.0, 3.0.0
>
>
> when the partition schema does not match table schema,it will thows exception
> when the task is running.For example,we modify the type of column from int to
> bigint by the sql *ALTER TABLE table_with_partition CHANGE COLUMN key key
> BIGINT* ,then we query the patition data which was stored before the
> changing,we would get the exception:
> {noformat}
> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in
> stage 27.0 failed 4 times, most recent failure: Lost task 0.3 in stage 27.0
> (TID 30, BJHC-HADOOP-HERA-16950.jeanlyn.local): java.lang.ClassCastException:
> org.apache.spark.sql.catalyst.expressions.MutableLong cannot be cast to
> org.apache.spark.sql.catalyst.expressions.MutableInt
> at
> org.apache.spark.sql.catalyst.expressions.SpecificMutableRow.setInt(SpecificMutableRow.scala:241)
> at
> org.apache.spark.sql.hive.HadoopTableReader$$anonfun$13$$anonfun$apply$4.apply(TableReader.scala:286)
> at
> org.apache.spark.sql.hive.HadoopTableReader$$anonfun$13$$anonfun$apply$4.apply(TableReader.scala:286)
> at
> org.apache.spark.sql.hive.HadoopTableReader$$anonfun$fillObject$1.apply(TableReader.scala:322)
> at
> org.apache.spark.sql.hive.HadoopTableReader$$anonfun$fillObject$1.apply(TableReader.scala:314)
> at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
> at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
> at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
> at scala.collection.Iterator$$anon$10.next(Iterator.scala:312)
> at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> at
> scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
> at
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
> at
> scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
> at
> scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
> at scala.collection.AbstractIterator.to(Iterator.scala:1157)
> at
> scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
> at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
> at
> scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
> at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
> at
> org.apache.spark.sql.execution.Limit$$anonfun$4.apply(basicOperators.scala:141)
> at
> org.apache.spark.sql.execution.Limit$$anonfun$4.apply(basicOperators.scala:141)
> at
> org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)
> at
> org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)
> at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
> at org.apache.spark.scheduler.Task.run(Task.scala:56)
> at
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)
> Driver stacktrace:
> at
> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1214)
> at
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1203)
> at
> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1202)
> at
> scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
> at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
> at
> org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1202)
> at
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:696)
> at
> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:696)
> at scala.Option.foreach(Option.scala:236)
> at
> org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:696)
> at
> org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1420)
> at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
> at
> org.apache.spark.scheduler.DAGSchedulerEventProcessActor.aroundReceive(DAGScheduler.scala:1375)
> at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> at akka.actor.ActorCell.invoke(ActorCell.scala:487)
> at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
> at akka.dispatch.Mailbox.run(Mailbox.scala:220)
> at
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:393)
> at
> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> at
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> at
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> at
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> {noformat}
> we can reproduce the bug as follow:
> add the code to the unit test
> *sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala*
> {noformat}
> test("partition schema does not match table schema"){
> val testData = TestHive.sparkContext.parallelize(
> (1 to 10).map(i => TestData(i, i.toString)))
> testData.registerTempTable("testData")
> val tmpDir = Files.createTempDir()
> sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED
> by (ds string) location '${tmpDir.toURI.toString}' ")
> sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1')
> SELECT key,value FROM testData")
> sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT")
> checkAnswer(sql("select key,value from table_with_partition where ds='1'
> "),
> testData.toSchemaRDD.collect.toSeq
> )
> sql("DROP TABLE table_with_partition")
>
> }
> {noformat}
> run the test
> {noformat}
> mvn -Dhadoop.version=... -
> DwildcardSuites=org.apache.spark.sql.hive.InsertIntoHiveTableSuite test
> {noformat}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]