You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Cheng Hao (JIRA)" <ji...@apache.org> on 2015/08/07 08:51:46 UTC

[jira] [Created] (SPARK-9735) Auto infer partition schema of HadoopFsRelation should should respected the user specified one

Cheng Hao created SPARK-9735:
--------------------------------

             Summary: Auto infer partition schema of HadoopFsRelation should should respected the user specified one
                 Key: SPARK-9735
                 URL: https://issues.apache.org/jira/browse/SPARK-9735
             Project: Spark
          Issue Type: Bug
          Components: SQL
            Reporter: Cheng Hao


This code is copied from the hadoopFsRelationSuite.scala

{code}
partitionedTestDF = (for {
    i <- 1 to 3
    p2 <- Seq("foo", "bar")
  } yield (i, s"val_$i", 1, p2)).toDF("a", "b", "p1", "p2")

  withTempPath { file =>
      val input = partitionedTestDF.select('a, 'b,    'p1.cast(StringType).as('ps), 'p2)

      input
        .write
        .format(dataSourceName)
        .mode(SaveMode.Overwrite)
        .partitionBy("ps", "p2")
        .saveAsTable("t")

      withTempTable("t") {
        checkAnswer(sqlContext.table("t"), input.collect())
      }
    }

11.521 ERROR org.apache.spark.executor.Executor: Exception in task 2.0 in stage 2.0 (TID 130)
java.lang.ClassCastException: java.lang.Integer cannot be cast to org.apache.spark.unsafe.types.UTF8String
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:45)
	at org.apache.spark.sql.catalyst.expressions.SpecificMutableRow.getUTF8String(SpecificMutableRow.scala:195)
	at org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toScalaImpl(CatalystTypeConverters.scala:297)
	at org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toScalaImpl(CatalystTypeConverters.scala:289)
	at org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toScala(CatalystTypeConverters.scala:110)
	at org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toScala(CatalystTypeConverters.scala:278)
	at org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toScala(CatalystTypeConverters.scala:245)
	at org.apache.spark.sql.catalyst.CatalystTypeConverters$$anonfun$createToScalaConverter$2.apply(CatalystTypeConverters.scala:406)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$3$$anonfun$apply$2.apply(SparkPlan.scala:194)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$3$$anonfun$apply$2.apply(SparkPlan.scala:194)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	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.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:905)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:905)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1836)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1836)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
{code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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