Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-3231

select on a table in parquet format containing smallint as a field type does not work

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.1.0
    • 1.5.0
    • SQL
    • None
    • The table is created through Hive-0.13.
      SparkSql 1.1 is used.

    Description

      A table is created through hive. This table has a field of type smallint. The format of the table is parquet.
      select on this table works perfectly on hive shell.
      But, when the select is run on this table from spark-sql, then the query fails.

      Steps to reproduce the issue:
      --------------------------------------
      hive> create table abct (a smallint, b int) row format delimited fields terminated by '|' stored as textfile;
      A text file is stored in hdfs for this table.

      hive> create table abc (a smallint, b int) stored as parquet;
      hive> insert overwrite table abc select * from abct;
      hive> select * from abc;
      2 1
      2 2
      2 3

      spark-sql> select * from abc;
      10:08:46 ERROR CliDriver: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0.0 in stage 33.0 (TID 2340) had a not serializable result: org.apache.hadoop.io.IntWritable
      at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1158)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1147)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1146)
      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:1146)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:685)
      at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:685)
      at scala.Option.foreach(Option.scala:236)
      at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:685)
      at org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1364)
      at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498)
      at akka.actor.ActorCell.invoke(ActorCell.scala:456)
      at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237)
      at akka.dispatch.Mailbox.run(Mailbox.scala:219)
      at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
      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)

      But, if the type of this table is now changed to int, then spark-sql gives the correct results.

      hive> alter table abc change a a int;
      spark-sql> select * from abc;

      2 1
      2 2
      2 3

      Attachments

        Activity

          People

            arov Alex Rovner
            chiragaggarwal chirag aggarwal
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: