Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/6617#discussion_r35720424
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala
 ---
    @@ -0,0 +1,565 @@
    +/*
    + * 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 scala.collection.JavaConversions._
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.parquet.schema.OriginalType._
    +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._
    +import org.apache.parquet.schema.Type.Repetition._
    +import org.apache.parquet.schema._
    +
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.{AnalysisException, SQLConf}
    +
    +/**
    + * This converter class is used to convert Parquet [[MessageType]] to 
Spark SQL [[StructType]] and
    + * vice versa.
    + *
    + * Parquet format backwards-compatibility rules are respected when 
converting Parquet
    + * [[MessageType]] schemas.
    + *
    + * @see 
https://github.com/apache/parquet-format/blob/master/LogicalTypes.md
    + *
    + * @constructor
    + * @param assumeBinaryIsString Whether unannotated BINARY fields should be 
assumed to be Spark SQL
    + *        [[StringType]] fields when converting Parquet a [[MessageType]] 
to Spark SQL
    + *        [[StructType]].
    + * @param assumeInt96IsTimestamp Whether unannotated INT96 fields should 
be assumed to be Spark SQL
    + *        [[TimestampType]] fields when converting Parquet a 
[[MessageType]] to Spark SQL
    + *        [[StructType]].  Note that Spark SQL [[TimestampType]] is 
similar to Hive timestamp, which
    + *        has optional nanosecond precision, but different from 
`TIME_MILLS` and `TIMESTAMP_MILLIS`
    + *        described in Parquet format spec.
    + * @param followParquetFormatSpec Whether to generate standard DECIMAL, 
LIST, and MAP structure when
    + *        converting Spark SQL [[StructType]] to Parquet [[MessageType]].  
For Spark 1.4.x and
    + *        prior versions, Spark SQL only supports decimals with a max 
precision of 18 digits, and
    + *        uses non-standard LIST and MAP structure.  Note that the current 
Parquet format spec is
    + *        backwards-compatible with these settings.  If this argument is 
set to `false`, we fallback
    + *        to old style non-standard behaviors.
    + */
    +private[parquet] class CatalystSchemaConverter(
    +    private val assumeBinaryIsString: Boolean,
    +    private val assumeInt96IsTimestamp: Boolean,
    +    private val followParquetFormatSpec: Boolean) {
    +
    +  // Only used when constructing converter for converting Spark SQL schema 
to Parquet schema, in
    +  // which case `assumeInt96IsTimestamp` and `assumeBinaryIsString` are 
irrelevant.
    +  def this() = this(
    +    assumeBinaryIsString = 
SQLConf.PARQUET_BINARY_AS_STRING.defaultValue.get,
    +    assumeInt96IsTimestamp = 
SQLConf.PARQUET_INT96_AS_TIMESTAMP.defaultValue.get,
    +    followParquetFormatSpec = 
SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.defaultValue.get)
    +
    +  def this(conf: SQLConf) = this(
    +    assumeBinaryIsString = conf.isParquetBinaryAsString,
    +    assumeInt96IsTimestamp = conf.isParquetINT96AsTimestamp,
    +    followParquetFormatSpec = conf.followParquetFormatSpec)
    +
    +  def this(conf: Configuration) = this(
    +    assumeBinaryIsString =
    +      conf.getBoolean(
    +        SQLConf.PARQUET_BINARY_AS_STRING.key,
    +        SQLConf.PARQUET_BINARY_AS_STRING.defaultValue.get),
    +    assumeInt96IsTimestamp =
    +      conf.getBoolean(
    +        SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
    +        SQLConf.PARQUET_INT96_AS_TIMESTAMP.defaultValue.get),
    +    followParquetFormatSpec =
    +      conf.getBoolean(
    +        SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.key,
    +        SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.defaultValue.get))
    +
    +  /**
    +   * Converts Parquet [[MessageType]] `parquetSchema` to a Spark SQL 
[[StructType]].
    +   */
    +  def convert(parquetSchema: MessageType): StructType = 
convert(parquetSchema.asGroupType())
    +
    +  private def convert(parquetSchema: GroupType): StructType = {
    +    val fields = parquetSchema.getFields.map { field =>
    +      field.getRepetition match {
    +        case OPTIONAL =>
    +          StructField(field.getName, convertField(field), nullable = true)
    +
    +        case REQUIRED =>
    +          StructField(field.getName, convertField(field), nullable = false)
    +
    +        case REPEATED =>
    +          throw new AnalysisException(
    +            s"REPEATED not supported outside LIST or MAP. Type: $field")
    +      }
    +    }
    +
    +    StructType(fields)
    +  }
    +
    +  /**
    +   * Converts a Parquet [[Type]] to a Spark SQL [[DataType]].
    +   */
    +  def convertField(parquetType: Type): DataType = parquetType match {
    +    case t: PrimitiveType => convertPrimitiveField(t)
    +    case t: GroupType => convertGroupField(t.asGroupType())
    +  }
    +
    +  private def convertPrimitiveField(field: PrimitiveType): DataType = {
    +    val typeName = field.getPrimitiveTypeName
    +    val originalType = field.getOriginalType
    +
    +    def typeString =
    +      if (originalType == null) s"$typeName" else s"$typeName 
($originalType)"
    +
    +    def typeNotImplemented() =
    +      throw new AnalysisException(s"Parquet type not yet supported: 
$typeString")
    +
    +    def illegalType() =
    +      throw new AnalysisException(s"Illegal Parquet type: $typeString")
    +
    +    // When maxPrecision = -1, we skip precision range check, and always 
respect the precision
    +    // specified in field.getDecimalMetadata.  This is useful when 
interpreting decimal types stored
    +    // as binaries with variable lengths.
    +    def makeDecimalType(maxPrecision: Int = -1): DecimalType = {
    +      val precision = field.getDecimalMetadata.getPrecision
    +      val scale = field.getDecimalMetadata.getScale
    +
    +      CatalystSchemaConverter.analysisRequire(
    +        maxPrecision == -1 || 1 <= precision && precision <= maxPrecision,
    +        s"Invalid decimal precision: $typeName cannot store $precision 
digits (max $maxPrecision)")
    +
    +      DecimalType(precision, scale)
    +    }
    +
    +    field.getPrimitiveTypeName match {
    +      case BOOLEAN => BooleanType
    +
    +      case FLOAT => FloatType
    +
    +      case DOUBLE => DoubleType
    +
    +      case INT32 =>
    +        field.getOriginalType match {
    +          case INT_8 => ByteType
    +          case INT_16 => ShortType
    +          case INT_32 | null => IntegerType
    +          case DATE => DateType
    +          case DECIMAL => makeDecimalType(maxPrecisionForBytes(4))
    +          case TIME_MILLIS => typeNotImplemented()
    +          case _ => illegalType()
    +        }
    +
    +      case INT64 =>
    +        field.getOriginalType match {
    +          case INT_64 | null => LongType
    +          case DECIMAL => makeDecimalType(maxPrecisionForBytes(8))
    +          case TIMESTAMP_MILLIS => typeNotImplemented()
    +          case _ => illegalType()
    +        }
    +
    +      case INT96 =>
    +        CatalystSchemaConverter.analysisRequire(
    +          assumeInt96IsTimestamp,
    +          "INT96 is not supported unless it's interpreted as timestamp. " +
    +            s"Please try to set ${SQLConf.PARQUET_INT96_AS_TIMESTAMP.key} 
to true.")
    +        TimestampType
    +
    +      case BINARY =>
    +        field.getOriginalType match {
    +          case UTF8 => StringType
    +          case null if assumeBinaryIsString => StringType
    +          case null => BinaryType
    +          case DECIMAL => makeDecimalType()
    +          case _ => illegalType()
    +        }
    +
    +      case FIXED_LEN_BYTE_ARRAY =>
    +        field.getOriginalType match {
    +          case DECIMAL => 
makeDecimalType(maxPrecisionForBytes(field.getTypeLength))
    +          case INTERVAL => typeNotImplemented()
    +          case _ => illegalType()
    +        }
    +
    +      case _ => illegalType()
    +    }
    +  }
    +
    +  private def convertGroupField(field: GroupType): DataType = {
    +    Option(field.getOriginalType).fold(convert(field): DataType) {
    +      // A Parquet list is represented as a 3-level structure:
    +      //
    +      //   <list-repetition> group <name> (LIST) {
    +      //     repeated group list {
    +      //       <element-repetition> <element-type> element;
    +      //     }
    +      //   }
    +      //
    +      // However, according to the most recent Parquet format spec (not 
released yet up until
    +      // writing), some 2-level structures are also recognized for 
backwards-compatibility.  Thus,
    +      // we need to check whether the 2nd level or the 3rd level refers to 
list element type.
    +      //
    +      // See: 
https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists
    +      case LIST =>
    +        CatalystSchemaConverter.analysisRequire(
    +          field.getFieldCount == 1, s"Invalid list type $field")
    +
    +        val repeatedType = field.getType(0)
    +        CatalystSchemaConverter.analysisRequire(
    +          repeatedType.isRepetition(REPEATED), s"Invalid list type $field")
    +
    +        if (isElementType(repeatedType, field.getName)) {
    +          ArrayType(convertField(repeatedType), containsNull = false)
    +        } else {
    +          val elementType = repeatedType.asGroupType().getType(0)
    +          val optional = elementType.isRepetition(OPTIONAL)
    +          ArrayType(convertField(elementType), containsNull = optional)
    +        }
    +
    +      // scalastyle:off
    +      // `MAP_KEY_VALUE` is for backwards-compatibility
    +      // See: 
https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules-1
    +      // scalastyle:on
    +      case MAP | MAP_KEY_VALUE =>
    +        CatalystSchemaConverter.analysisRequire(
    +          field.getFieldCount == 1 && !field.getType(0).isPrimitive,
    +          s"Invalid map type: $field")
    +
    +        val keyValueType = field.getType(0).asGroupType()
    +        CatalystSchemaConverter.analysisRequire(
    +          keyValueType.isRepetition(REPEATED) && 
keyValueType.getFieldCount == 2,
    +          s"Invalid map type: $field")
    +
    +        val keyType = keyValueType.getType(0)
    +        CatalystSchemaConverter.analysisRequire(
    +          keyType.isPrimitive,
    +          s"Map key type is expected to be a primitive type, but found: 
$keyType")
    +
    +        val valueType = keyValueType.getType(1)
    +        val valueOptional = valueType.isRepetition(OPTIONAL)
    +        MapType(
    +          convertField(keyType),
    +          convertField(valueType),
    +          valueContainsNull = valueOptional)
    +
    +      case _ =>
    +        throw new AnalysisException(s"Unrecognized Parquet type: $field")
    +    }
    +  }
    +
    +  // scalastyle:off
    +  // Here we implement Parquet LIST backwards-compatibility rules.
    +  // See: 
https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules
    +  // scalastyle:on
    +  private def isElementType(repeatedType: Type, parentName: String) = {
    +    {
    +      // For legacy 2-level list types with primitive element type, e.g.:
    +      //
    +      //    // List<Integer> (nullable list, non-null elements)
    +      //    optional group my_list (LIST) {
    +      //      repeated int32 element;
    +      //    }
    +      //
    +      repeatedType.isPrimitive
    +    } || {
    +      // For legacy 2-level list types whose element type is a group type 
with 2 or more fields,
    +      // e.g.:
    +      //
    +      //    // List<Tuple<String, Integer>> (nullable list, non-null 
elements)
    +      //    optional group my_list (LIST) {
    +      //      repeated group element {
    +      //        required binary str (UTF8);
    +      //        required int32 num;
    +      //      };
    +      //    }
    +      //
    +      repeatedType.asGroupType().getFieldCount > 1
    +    } || {
    +      // For legacy 2-level list types generated by parquet-avro (Parquet 
version < 1.6.0), e.g.:
    +      //
    +      //    // List<OneTuple<String>> (nullable list, non-null elements)
    +      //    optional group my_list (LIST) {
    +      //      repeated group array {
    +      //        required binary str (UTF8);
    +      //      };
    +      //    }
    +      //
    +      repeatedType.getName == "array"
    +    } || {
    +      // For Parquet data generated by parquet-thrift, e.g.:
    +      //
    +      //    // List<OneTuple<String>> (nullable list, non-null elements)
    +      //    optional group my_list (LIST) {
    +      //      repeated group my_list_tuple {
    +      //        required binary str (UTF8);
    +      //      };
    +      //    }
    +      //
    +      repeatedType.getName == s"${parentName}_tuple"
    +    }
    +  }
    +
    +  /**
    +   * Converts a Spark SQL [[StructType]] to a Parquet [[MessageType]].
    +   */
    +  def convert(catalystSchema: StructType): MessageType = {
    +    Types.buildMessage().addFields(catalystSchema.map(convertField): 
_*).named("root")
    --- End diff --
    
    I believe you were referring to [this line] [1] in `HiveSchemaConverter`. 
Using a more explicit name makes sense to me. Will include this change in 
#7679. Thanks!
    
    [1]: 
https://github.com/apache/parquet-mr/blob/master/parquet-hive/parquet-hive-storage-handler/src/main/java/org/apache/hadoop/hive/ql/io/parquet/convert/HiveSchemaConverter.java#L43


---
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.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to