Github user AndreSchumacher commented on a diff in the pull request:
https://github.com/apache/spark/pull/360#discussion_r11986397
--- Diff:
sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala ---
@@ -0,0 +1,369 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.parquet
+
+import java.io.IOException
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.mapreduce.Job
+
+import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter}
+import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData}
+import parquet.hadoop.util.ContextUtil
+import parquet.schema.{Type => ParquetType, PrimitiveType =>
ParquetPrimitiveType, MessageType, MessageTypeParser}
+import parquet.schema.{GroupType => ParquetGroupType, OriginalType =>
ParquetOriginalType, ConversionPatterns}
+import parquet.schema.PrimitiveType.{PrimitiveTypeName =>
ParquetPrimitiveTypeName}
+import parquet.schema.Type.Repetition
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference,
Attribute}
+import org.apache.spark.sql.catalyst.types._
+
+// Implicits
+import scala.collection.JavaConversions._
+
+private[parquet] object ParquetTypesConverter {
+ def isPrimitiveType(ctype: DataType): Boolean =
+ classOf[PrimitiveType] isAssignableFrom ctype.getClass
+
+ def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName):
DataType = parquetType match {
+ case ParquetPrimitiveTypeName.BINARY => StringType
+ case ParquetPrimitiveTypeName.BOOLEAN => BooleanType
+ case ParquetPrimitiveTypeName.DOUBLE => DoubleType
+ case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY =>
ArrayType(ByteType)
+ case ParquetPrimitiveTypeName.FLOAT => FloatType
+ case ParquetPrimitiveTypeName.INT32 => IntegerType
+ case ParquetPrimitiveTypeName.INT64 => LongType
+ case ParquetPrimitiveTypeName.INT96 =>
+ // TODO: add BigInteger type? TODO(andre) use DecimalType instead????
+ sys.error("Potential loss of precision: cannot convert INT96")
+ case _ => sys.error(
+ s"Unsupported parquet datatype $parquetType")
+ }
+
+ /**
+ * Converts a given Parquet `Type` into the corresponding
+ * [[org.apache.spark.sql.catalyst.types.DataType]].
+ *
+ * Note that we apply the following conversion rules:
+ * <ul>
+ * <li> Primitive types are converter to the corresponding primitive
type.</li>
+ * <li> Group types that have a single field that is itself a group,
which has repetition
+ * level `REPEATED`, are treated as follows:<ul>
+ * <li> If the nested group has name `values`, the surrounding
group is converted
+ * into an [[ArrayType]] with the corresponding field type
(primitive or
+ * complex) as element type.</li>
+ * <li> If the nested group has name `map` and two fields
(named `key` and `value`),
+ * the surrounding group is converted into a [[MapType]]
+ * with the corresponding key and value (value possibly
complex) types.
+ * Note that we currently assume map values are not
nullable.</li>
+ * <li> Other group types are converted into a [[StructType]] with the
corresponding
+ * field types.</li></ul></li>
+ * </ul>
+ * Note that fields are determined to be `nullable` if and only if their
Parquet repetition
+ * level is not `REQUIRED`.
+ *
+ * @param parquetType The type to convert.
+ * @return The corresponding Catalyst type.
+ */
+ def toDataType(parquetType: ParquetType): DataType = {
+ def correspondsToMap(groupType: ParquetGroupType): Boolean = {
+ if (groupType.getFieldCount != 1 ||
groupType.getFields.apply(0).isPrimitive) {
+ false
+ } else {
+ // This mostly follows the convention in
``parquet.schema.ConversionPatterns``
--- End diff --
Bad wording, sorry. It comes from both `ConversionPatterns` and the way
Avro records are converted from and to Parquet. If one wants to be able to read
ParquetAvro one needs to follow these because otherwise one ends up having
parsing errors. I will improve that comment.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---