From 2e729797c677e2268e1ec3cd26c9eefd4a6e245d Mon Sep 17 00:00:00 2001 From: Steven She Date: Fri, 26 Jun 2015 15:30:34 -0400 Subject: [PATCH] [SPARK-8669] [SQL] Fix crash with BINARY (ENUM) fields with Parquet 1.7 Always converts BINARY (ENUM) to StringType regardless of assumeBinaryIsString. --- .../spark/sql/parquet/CatalystSchemaConverter.scala | 2 +- .../org/apache/spark/sql/parquet/ParquetSchemaSuite.scala | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala index 4fd3e93b70311..2be7c64612cd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala @@ -177,7 +177,7 @@ private[parquet] class CatalystSchemaConverter( case BINARY => field.getOriginalType match { - case UTF8 => StringType + case UTF8 | ENUM => StringType case null if assumeBinaryIsString => StringType case null => BinaryType case DECIMAL => makeDecimalType() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index d0bfcde7e032b..35d3c33f99a06 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -161,6 +161,14 @@ class ParquetSchemaInferenceSuite extends ParquetSchemaTest { """.stripMargin, binaryAsString = true) + testSchemaInference[Tuple1[String]]( + "binary enum as string", + """ + |message root { + | optional binary _1 (ENUM); + |} + """.stripMargin) + testSchemaInference[Tuple1[Seq[Int]]]( "non-nullable array - non-standard", """