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

Reading tables partitioned by columns that look like timestamps has inconsistent schema inference

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • 2.2.0
    • 2.2.1, 2.3.0
    • SQL
    • None

    Description

      If you try to read a partitioned json table, spark automatically tries to read figure out if the partition column is a timestamp based on the first value it sees. So if you really partitioned by a string, and the first value happens to look like a timestamp, then you'll run into errors. Even if you specify a schema, the schema is ignored, and spark still tries to infer a timestamp type for the partition column.

      This is particularly weird because schema-inference does not work for regular timestamp columns in a flat table. You have to manually specify the schema to get the column interpreted as a timestamp.

      This problem does not appear to be present for other types. Eg., if I partition by a string column, and the first value happens to look like an int, schema inference is still fine.

      Here's a small example:

      val df = Seq(
        (1, "2015-01-01 00:00:00", Timestamp.valueOf("2015-01-01 00:00:00")),
        (2, "2014-01-01 00:00:00", Timestamp.valueOf("2014-01-01 00:00:00")),
        (3, "blah", Timestamp.valueOf("2016-01-01 00:00:00"))).toDF("i", "str", "t")
      
      
      df.write.partitionBy("str").json("partition_by_str")
      df.write.partitionBy("t").json("partition_by_t")
      df.write.json("flat")
      
      val readStr = spark.read.json("partition_by_str")/*
      java.util.NoSuchElementException: None.get
        at scala.None$.get(Option.scala:347)
        at scala.None$.get(Option.scala:345)
        at org.apache.spark.sql.catalyst.expressions.TimeZoneAwareExpression$class.timeZone(datetimeExpressions.scala:46)
        at org.apache.spark.sql.catalyst.expressions.Cast.timeZone$lzycompute(Cast.scala:172)
        at org.apache.spark.sql.catalyst.expressions.Cast.timeZone(Cast.scala:172)
        at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3$$anonfun$apply$16.apply(Cast.scala:208)  at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3$$anonfun$apply$16.apply(Cast.scala:208)
        at org.apache.spark.sql.catalyst.expressions.Cast.org$apache$spark$sql$catalyst$expressions$Cast$$buildCast(Cast.scala:201)
        at org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToString$3.apply(Cast.scala:207)
        at org.apache.spark.sql.catalyst.expressions.Cast.nullSafeEval(Cast.scala:533)
        at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:327)
        at org.apache.spark.sql.execution.datasources.PartitioningUtils$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningUtils$$resolveTypeConflicts$1.apply(PartitioningUtils.scala:485)
        at org.apache.spark.sql.execution.datasources.PartitioningUtils$$anonfun$org$apache$spark$sql$execution$datasources$PartitioningUtils$$resolveTypeConflicts$1.apply(PartitioningUtils.scala:484)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
        at scala.collection.AbstractTraversable.map(Traversable.scala:104)
        at org.apache.spark.sql.execution.datasources.PartitioningUtils$.org$apache$spark$sql$execution$datasources$PartitioningUtils$$resolveTypeConflicts(PartitioningUtils.scala:484)
        at org.apache.spark.sql.execution.datasources.PartitioningUtils$$anonfun$15.apply(PartitioningUtils.scala:340)
        at org.apache.spark.sql.execution.datasources.PartitioningUtils$$anonfun$15.apply(PartitioningUtils.scala:339)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at scala.collection.immutable.Range.foreach(Range.scala:160)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
        at scala.collection.AbstractTraversable.map(Traversable.scala:104)
        at org.apache.spark.sql.execution.datasources.PartitioningUtils$.resolvePartitions(PartitioningUtils.scala:339)
        at org.apache.spark.sql.execution.datasources.PartitioningUtils$.parsePartitions(PartitioningUtils.scala:141)
        at org.apache.spark.sql.execution.datasources.PartitioningUtils$.parsePartitions(PartitioningUtils.scala:97)
        at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.inferPartitioning(PartitioningAwareFileIndex.scala:153)
        at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.partitionSpec(InMemoryFileIndex.scala:70)
        at org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex.partitionSchema(PartitioningAwareFileIndex.scala:50)
        at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:133)
        at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:366)
        at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
        at org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:333)
        at org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:279)
        ... 48 elided
      */
      
      val readStr = spark.read.schema(df.schema).json("partition_by_str")
      /*
      same exception
      */
      
      val readT = spark.read.json("partition_by_t") // OK
      val readT = spark.read.schema(df.schema).json("partition_by_t") // OK
      
      val readFlat = spark.read.json("flat") // NO error, by timestamp column is read a String
      val readFlat = spark.read.schema(df.schema).json("flat") // OK
      

      Attachments

        Activity

          People

            gurwls223 Hyukjin Kwon
            irashid Imran Rashid
            Votes:
            1 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: