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