From 03c3bd9184b6392316c2c8bee03a312bcfcdf7b7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 4 Jul 2015 23:04:35 -0700 Subject: [PATCH 01/25] Refactors Parquet read path to implement backwards-compatibility rules --- .../sql/catalyst/util/DateTimeUtils.scala | 3 +- .../org/apache/spark/sql/types/DataType.scala | 6 +- .../apache/spark/sql/types/StructType.scala | 5 + .../sql/parquet/CatalystRowConverter.scala | 423 ++++++++++++++++++ .../sql/parquet/ParquetTableSupport.scala | 112 ++--- .../apache/spark/sql/parquet/newParquet.scala | 22 +- .../spark/sql/parquet/ParquetIOSuite.scala | 7 +- .../spark/sql/parquet/ParquetTest.scala | 2 +- 8 files changed, 507 insertions(+), 73 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 4269ad5d56737..b244c470077bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -29,11 +29,10 @@ import java.util.{Calendar, TimeZone} * precision. */ object DateTimeUtils { - final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L - // see http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian final val JULIAN_DAY_OF_EPOCH = 2440587 // and .5 final val SECONDS_PER_DAY = 60 * 60 * 24L + final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L final val HUNDRED_NANOS_PER_SECOND = 1000L * 1000L * 10L final val NANOS_PER_SECOND = HUNDRED_NANOS_PER_SECOND * 100 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 7d00047d08d74..b0fdbecdaa5a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -17,11 +17,12 @@ package org.apache.spark.sql.types +import scala.util.Try import scala.util.parsing.combinator.RegexParsers -import org.json4s._ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ +import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.DeveloperApi @@ -82,6 +83,9 @@ abstract class DataType extends AbstractDataType { object DataType { + private[sql] def fromString(raw: String) = { + Try(DataType.fromJson(raw)).getOrElse(DataType.fromCaseClassString(raw)) + } def fromJson(json: String): DataType = parseDataType(parse(json)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 3b17566d54d9b..e2d3f53f7d978 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -311,6 +311,11 @@ object StructType extends AbstractDataType { private[sql] override def simpleString: String = "struct" + private[sql] def fromString(raw: String): StructType = DataType.fromString(raw) match { + case t: StructType => t + case _ => throw new RuntimeException(s"Failed parsing StructType: $raw") + } + def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) def apply(fields: java.util.List[StructField]): StructType = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala new file mode 100644 index 0000000000000..b7b0c6c782f3a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala @@ -0,0 +1,423 @@ +/* + * 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.nio.ByteOrder + +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.parquet.column.Dictionary +import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter} +import org.apache.parquet.schema.Type.Repetition +import org.apache.parquet.schema.{GroupType, PrimitiveType, Type} + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** + * A [[ParentContainerUpdater]] is used by a Parquet converter to set converted values to some + * corresponding parent container. For example, a converter for a `StructType` field may set + * converted values to a [[MutableRow]]; or a converter for array elements may append converted + * values to an [[ArrayBuffer]]. + */ +private[parquet] trait ParentContainerUpdater { + def set(value: Any): Unit = () + def setBoolean(value: Boolean): Unit = set(value) + def setByte(value: Byte): Unit = set(value) + def setShort(value: Short): Unit = set(value) + def setInt(value: Int): Unit = set(value) + def setLong(value: Long): Unit = set(value) + def setFloat(value: Float): Unit = set(value) + def setDouble(value: Double): Unit = set(value) +} + +/** A no-op updater used for root converter (who doesn't have a parent). */ +private[parquet] object NoopUpdater extends ParentContainerUpdater + +/** + * This Parquet converter converts Parquet records to Spark SQL [[Row]]s. + * + * @param parquetType Parquet schema of Parquet records + * @param catalystType Spark SQL schema that corresponds to the Parquet record type + * @param updater An updater which takes care of the converted row object + */ +private[parquet] class CatalystRowConverter( + parquetType: GroupType, + catalystType: StructType, + updater: ParentContainerUpdater) + extends GroupConverter { + + /** + * Updater used together with field converters of [[CatalystRowConverter]]. It sets converted + * filed values to the `ordinal`-th cell in `currentRow`. + */ + private final class RowUpdater(row: MutableRow, ordinal: Int) extends ParentContainerUpdater { + override def set(value: Any): Unit = row(ordinal) = value + override def setBoolean(value: Boolean): Unit = row.setBoolean(ordinal, value) + override def setByte(value: Byte): Unit = row.setByte(ordinal, value) + override def setShort(value: Short): Unit = row.setShort(ordinal, value) + override def setInt(value: Int): Unit = row.setInt(ordinal, value) + override def setLong(value: Long): Unit = row.setLong(ordinal, value) + override def setDouble(value: Double): Unit = row.setDouble(ordinal, value) + override def setFloat(value: Float): Unit = row.setFloat(ordinal, value) + } + + /** + * Represents the converted row object once an entire Parquet record is converted. + * + * @todo Uses [[UnsafeRow]] for better performance. + */ + val currentRow = new SpecificMutableRow(catalystType.map(_.dataType)) + + // Converters for each field. + private val fieldConverters: Array[Converter] = { + parquetType.getFields.zip(catalystType).zipWithIndex.map { + case ((parquetFieldType, catalystField), ordinal) => + // Converted field value should be set to the `ordinal`-th cell of `currentRow` + newConverter(parquetFieldType, catalystField.dataType, new RowUpdater(currentRow, ordinal)) + }.toArray + } + + override def getConverter(fieldIndex: Int): Converter = fieldConverters(fieldIndex) + + override def end(): Unit = updater.set(currentRow) + + override def start(): Unit = { + var i = 0 + while (i < currentRow.length) { + currentRow.setNullAt(i) + i += 1 + } + } + + /** + * Creates a converter for the given Parquet type `parquetType` and Spark SQL data type + * `catalystType`. Converted values are handled by `updater`. + */ + private def newConverter( + parquetType: Type, + catalystType: DataType, + updater: ParentContainerUpdater): Converter = { + + catalystType match { + case BooleanType | IntegerType | LongType | FloatType | DoubleType | BinaryType => + new CatalystPrimitiveConverter(updater) + + case ByteType => + new PrimitiveConverter { + override def addInt(value: Int): Unit = + updater.setByte(value.asInstanceOf[ByteType#InternalType]) + } + + case ShortType => + new PrimitiveConverter { + override def addInt(value: Int): Unit = + updater.setShort(value.asInstanceOf[ShortType#InternalType]) + } + + case t: DecimalType => + new CatalystDecimalConverter(t, updater) + + case StringType => + new CatalystStringConverter(updater) + + case TimestampType => + new PrimitiveConverter { + override def addBinary(value: Binary): Unit = { + assert( + value.length() == 12, + "Timestamps (with nanoseconds) are expected to be stored in 12-byte long binaries, " + + s"but got a ${value.length()}-byte binary.") + + val buf = value.toByteBuffer.order(ByteOrder.LITTLE_ENDIAN) + val timeOfDayNanos = buf.getLong + val julianDay = buf.getInt + updater.setLong(DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos)) + } + } + + case DateType => + new PrimitiveConverter { + override def addInt(value: Int): Unit = { + // DateType is not specialized in `SpecificMutableRow`, have to box it here. + updater.set(value.asInstanceOf[DateType#InternalType]) + } + } + + case t: ArrayType => + new CatalystArrayConverter(parquetType.asGroupType(), t, updater) + + case t: MapType => + new CatalystMapConverter(parquetType.asGroupType(), t, updater) + + case t: StructType => + new CatalystRowConverter(parquetType.asGroupType(), t, updater) + + case t: UserDefinedType[_] => + val catalystTypeForUDT = t.sqlType + val nullable = parquetType.isRepetition(Repetition.OPTIONAL) + val field = StructField("udt", catalystTypeForUDT, nullable) + val parquetTypeForUDT = new CatalystSchemaConverter().convertField(field) + newConverter(parquetTypeForUDT, catalystTypeForUDT, updater) + + case _ => + throw new RuntimeException( + s"Unable to create Parquet converter for data type ${catalystType.json}") + } + } + + /** + * Parquet converter for Parquet primitive types. Note that not all Spark SQL primitive types + * are handled by this converter. Parquet primitive types are only a subset of those of Spark + * SQL. For example, BYTE, SHORT and INT in Spark SQL are all covered by INT32 in Parquet. + */ + private final class CatalystPrimitiveConverter(updater: ParentContainerUpdater) + extends PrimitiveConverter { + + override def addBoolean(value: Boolean): Unit = updater.setBoolean(value) + override def addInt(value: Int): Unit = updater.setInt(value) + override def addLong(value: Long): Unit = updater.setLong(value) + override def addFloat(value: Float): Unit = updater.setFloat(value) + override def addDouble(value: Double): Unit = updater.setDouble(value) + override def addBinary(value: Binary): Unit = updater.set(value.getBytes) + } + + /** + * Parquet converter for strings. A dictionary is used to minimize string decoding cost. + */ + private final class CatalystStringConverter(updater: ParentContainerUpdater) + extends PrimitiveConverter { + + private var expandedDictionary: Array[UTF8String] = null + + override def hasDictionarySupport: Boolean = true + + override def setDictionary(dictionary: Dictionary): Unit = { + this.expandedDictionary = Array.tabulate(dictionary.getMaxId + 1) { i => + UTF8String.fromBytes(dictionary.decodeToBinary(i).getBytes) + } + } + + override def addValueFromDictionary(dictionaryId: Int): Unit = { + updater.set(expandedDictionary(dictionaryId)) + } + + override def addBinary(value: Binary): Unit = { + updater.set(UTF8String.fromBytes(value.getBytes)) + } + } + + /** + * Parquet converter for fixed-precision decimals. + */ + private final class CatalystDecimalConverter( + decimalType: DecimalType, + updater: ParentContainerUpdater) + extends PrimitiveConverter { + + override def addInt(value: Int): Unit = { + addLong(value: Long) + } + + override def addLong(value: Long): Unit = { + updater.set(Decimal(value, decimalType.precision, decimalType.scale)) + } + + override def addBinary(value: Binary): Unit = { + updater.set(toDecimal(value)) + } + + private def toDecimal(value: Binary): Decimal = { + val precision = decimalType.precision + val scale = decimalType.scale + val bytes = value.getBytes + + require(bytes.length <= 16, "Decimal field too large to read") + + var unscaled = 0L + var i = 0 + + while (i < bytes.length) { + unscaled = (unscaled << 8) | (bytes(i) & 0xff) + i += 1 + } + + val bits = 8 * bytes.length + unscaled = (unscaled << (64 - bits)) >> (64 - bits) + Decimal(unscaled, precision, scale) + } + } + + /** + * Parquet converter for arrays. Spark SQL arrays are represented as Parquet lists. Standard + * Parquet lists are represented as a 3-level group annotated by `LIST`: + * {{{ + * group (LIST) { <-- parquetSchema points here + * repeated group list { + * element; + * } + * } + * }}} + * The `parquetSchema` constructor argument points to the outermost group. + * + * However, before this representation is standardized, some Parquet libraries/tools also use some + * non-standard formats to represent list-like structures. Backwards-compatibility rules for + * handling these cases are described in Parquet format spec. + * + * @see https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists + */ + private final class CatalystArrayConverter( + parquetSchema: GroupType, + catalystSchema: ArrayType, + updater: ParentContainerUpdater) + extends GroupConverter { + + private var currentArray: ArrayBuffer[Any] = _ + + private val elementConverter: Converter = { + val repeatedType = parquetSchema.getType(0) + val elementType = catalystSchema.elementType + + if (isElementType(repeatedType, elementType)) { + newConverter(repeatedType, elementType, new ParentContainerUpdater { + override def set(value: Any): Unit = currentArray += value + }) + } else { + new ElementConverter(repeatedType.asGroupType().getType(0), elementType) + } + } + + override def getConverter(fieldIndex: Int): Converter = elementConverter + + override def end(): Unit = updater.set(currentArray) + + override def start(): Unit = { + currentArray = ArrayBuffer.empty[Any] + } + + // scalastyle:off + /** + * Returns whether the given type is the element type of a list or is a syntactic group with + * one field that is the element type. This is determined by checking whether the type can be + * a syntactic group and by checking whether a potential syntactic group matches the expected + * schema. + * {{{ + * group (LIST) { + * repeated group list { <-- repeatedType points here + * element; + * } + * } + * }}} + * In short, here we handle Parquet list backwards-compatibility rules on the read path. This + * method is based on `AvroIndexedRecordConverter.isElementType`. + * + * @see https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules + */ + // scalastyle:on + private def isElementType(repeatedType: Type, elementType: DataType): Boolean = { + (repeatedType, elementType) match { + case (t: PrimitiveType, _) => true + case (t: GroupType, _) if t.getFieldCount > 1 => true + case (t: GroupType, StructType(Array(f))) if f.name == t.getFieldName(0) => true + case _ => false + } + } + + /** Array element converter */ + private final class ElementConverter(parquetType: Type, catalystType: DataType) + extends GroupConverter { + + private var currentElement: Any = _ + + private val converter = newConverter(parquetType, catalystType, new ParentContainerUpdater { + override def set(value: Any): Unit = currentElement = value + }) + + override def getConverter(fieldIndex: Int): Converter = converter + + override def end(): Unit = currentArray += currentElement + + override def start(): Unit = currentElement = null + } + } + + /** Parquet converter for maps */ + private final class CatalystMapConverter( + parquetType: GroupType, + catalystType: MapType, + updater: ParentContainerUpdater) + extends GroupConverter { + + private var currentMap: mutable.Map[Any, Any] = _ + + private val keyValueConverter = { + val repeatedType = parquetType.getType(0).asGroupType() + new KeyValueConverter( + repeatedType.getType(0), + repeatedType.getType(1), + catalystType.keyType, + catalystType.valueType) + } + + override def getConverter(fieldIndex: Int): Converter = keyValueConverter + + override def end(): Unit = updater.set(currentMap) + + override def start(): Unit = { + currentMap = mutable.Map.empty[Any, Any] + } + + /** Parquet converter for key-value pairs within the map. */ + private final class KeyValueConverter( + parquetKeyType: Type, + parquetValueType: Type, + catalystKeyType: DataType, + catalystValueType: DataType) + extends GroupConverter { + + private var currentKey: Any = _ + + private var currentValue: Any = _ + + private val converters = Array( + // Converter for keys + newConverter(parquetKeyType, catalystKeyType, new ParentContainerUpdater { + override def set(value: Any): Unit = currentKey = value + }), + + // Converter for values + newConverter(parquetValueType, catalystValueType, new ParentContainerUpdater { + override def set(value: Any): Unit = currentValue = value + })) + + override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) + + override def end(): Unit = currentMap(currentKey) = currentValue + + override def start(): Unit = { + currentKey = null + currentValue = null + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 8402cd756140d..67578dc88f48f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -17,14 +17,17 @@ package org.apache.spark.sql.parquet -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} +import java.util import java.util.{HashMap => JHashMap} +import scala.collection.JavaConversions._ + import org.apache.hadoop.conf.Configuration import org.apache.parquet.column.ParquetProperties import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.parquet.hadoop.api.ReadSupport.ReadContext -import org.apache.parquet.hadoop.api.{ReadSupport, WriteSupport} +import org.apache.parquet.hadoop.api.{InitContext, ReadSupport, WriteSupport} import org.apache.parquet.io.api._ import org.apache.parquet.schema.MessageType @@ -36,87 +39,68 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String /** - * A `parquet.io.api.RecordMaterializer` for Rows. + * A [[RecordMaterializer]] for Catalyst rows. * - *@param root The root group converter for the record. + * @param parquetSchema Parquet schema of the records to be read + * @param catalystSchema Catalyst schema of the rows to be constructed */ -private[parquet] class RowRecordMaterializer(root: CatalystConverter) +private[parquet] class RowRecordMaterializer(parquetSchema: MessageType, catalystSchema: StructType) extends RecordMaterializer[InternalRow] { - def this(parquetSchema: MessageType, attributes: Seq[Attribute]) = - this(CatalystConverter.createRootConverter(parquetSchema, attributes)) + private val rootConverter = new CatalystRowConverter(parquetSchema, catalystSchema, NoopUpdater) - override def getCurrentRecord: InternalRow = root.getCurrentRecord + override def getCurrentRecord: InternalRow = rootConverter.currentRow - override def getRootConverter: GroupConverter = root.asInstanceOf[GroupConverter] + override def getRootConverter: GroupConverter = rootConverter } -/** - * A `parquet.hadoop.api.ReadSupport` for Row objects. - */ private[parquet] class RowReadSupport extends ReadSupport[InternalRow] with Logging { - override def prepareForRead( conf: Configuration, - stringMap: java.util.Map[String, String], + keyValueMetaData: util.Map[String, String], fileSchema: MessageType, readContext: ReadContext): RecordMaterializer[InternalRow] = { - log.debug(s"preparing for read with Parquet file schema $fileSchema") - // Note: this very much imitates AvroParquet + log.debug(s"Preparing for read Parquet file with message type: $fileSchema") + + val toCatalyst = new CatalystSchemaConverter(conf) val parquetSchema = readContext.getRequestedSchema - var schema: Seq[Attribute] = null - - if (readContext.getReadSupportMetadata != null) { - // first try to find the read schema inside the metadata (can result from projections) - if ( - readContext - .getReadSupportMetadata - .get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA) != null) { - schema = ParquetTypesConverter.convertFromString( - readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA)) - } else { - // if unavailable, try the schema that was read originally from the file or provided - // during the creation of the Parquet relation - if (readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_METADATA_KEY) != null) { - schema = ParquetTypesConverter.convertFromString( - readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) + val catalystSchema = + Option(readContext.getReadSupportMetadata) + .map(_.toMap) + .flatMap { metadata => + metadata + // First tries to read requested schema, which may result from projections + .get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA) + // If not available, tries to read Catalyst schema from file metadata. It's only + // available if the target file is written by Spark SQL. + .orElse(metadata.get(RowReadSupport.SPARK_METADATA_KEY)) } - } - } - // if both unavailable, fall back to deducing the schema from the given Parquet schema - // TODO: Why it can be null? - if (schema == null) { - log.debug("falling back to Parquet read schema") - schema = ParquetTypesConverter.convertToAttributes(parquetSchema, false, true) - } - log.debug(s"list of attributes that will be read: $schema") - new RowRecordMaterializer(parquetSchema, schema) + .map(StructType.fromString) + .getOrElse { + logDebug("Catalyst schema not available, falling back to Parquet message type") + toCatalyst.convert(parquetSchema) + } + + logDebug(s"Catalyst schema used to read Parquet files: $catalystSchema") + new RowRecordMaterializer(parquetSchema, catalystSchema) } - override def init( - configuration: Configuration, - keyValueMetaData: java.util.Map[String, String], - fileSchema: MessageType): ReadContext = { - var parquetSchema = fileSchema - val metadata = new JHashMap[String, String]() - val requestedAttributes = RowReadSupport.getRequestedSchema(configuration) - - if (requestedAttributes != null) { - // If the parquet file is thrift derived, there is a good chance that - // it will have the thrift class in metadata. - val isThriftDerived = keyValueMetaData.keySet().contains("thrift.class") - parquetSchema = ParquetTypesConverter.convertFromAttributes(requestedAttributes) - metadata.put( - RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertToString(requestedAttributes)) - } + override def init(context: InitContext): ReadContext = { + val conf = context.getConfiguration + val maybeRowSchema = Option(conf.get(RowWriteSupport.SPARK_ROW_SCHEMA)) + val maybeRequestedSchema = Option(conf.get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA)) - val origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) - if (origAttributesStr != null) { - metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) - } + val parquetSchema = maybeRequestedSchema.map { schemaString => + val toParquet = new CatalystSchemaConverter(conf) + toParquet.convert(StructType.fromString(schemaString)) + }.getOrElse(context.getFileSchema) + + val metadata = + Map.empty[String, String] ++ + maybeRequestedSchema.map(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA -> _) ++ + maybeRowSchema.map(RowWriteSupport.SPARK_ROW_SCHEMA -> _) - new ReadSupport.ReadContext(parquetSchema, metadata) + new ReadContext(parquetSchema, metadata) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 6bc69c6ad0847..5464624e58e2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -257,6 +257,10 @@ private[sql] class ParquetRelation2( broadcastedConf: Broadcast[SerializableConfiguration]): RDD[Row] = { val useMetadataCache = sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA) val parquetFilterPushDown = sqlContext.conf.parquetFilterPushDown + val assumeBinaryIsString = sqlContext.conf.isParquetBinaryAsString + val assumeInt96IsTimestamp = sqlContext.conf.isParquetINT96AsTimestamp + val followParquetFormatSpec = sqlContext.conf.followParquetFormatSpec + // Create the function to set variable Parquet confs at both driver and executor side. val initLocalJobFuncOpt = ParquetRelation2.initializeLocalJobFunc( @@ -264,7 +268,11 @@ private[sql] class ParquetRelation2( filters, dataSchema, useMetadataCache, - parquetFilterPushDown) _ + parquetFilterPushDown, + assumeBinaryIsString, + assumeInt96IsTimestamp, + followParquetFormatSpec) _ + // Create the function to set input paths at the driver side. val setInputPaths = ParquetRelation2.initializeDriverSideJobFunc(inputFiles) _ @@ -469,9 +477,12 @@ private[sql] object ParquetRelation2 extends Logging { filters: Array[Filter], dataSchema: StructType, useMetadataCache: Boolean, - parquetFilterPushDown: Boolean)(job: Job): Unit = { + parquetFilterPushDown: Boolean, + assumeBinaryIsString: Boolean, + assumeInt96IsTimestamp: Boolean, + followParquetFormatSpec: Boolean)(job: Job): Unit = { val conf = job.getConfiguration - conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[RowReadSupport].getName()) + conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[RowReadSupport].getName) // Try to push down filters when filter push-down is enabled. if (parquetFilterPushDown) { @@ -495,6 +506,11 @@ private[sql] object ParquetRelation2 extends Logging { // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata conf.setBoolean(SQLConf.PARQUET_CACHE_METADATA.key, useMetadataCache) + + // Sets flags for Parquet schema conversion + conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, assumeBinaryIsString) + conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, assumeInt96IsTimestamp) + conf.setBoolean(SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.key, followParquetFormatSpec) } /** This closure sets input paths at the driver side. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 7b16eba00d6fb..c5bce6e163a35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -71,7 +71,10 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { * Writes `data` to a Parquet file, reads it back and check file contents. */ protected def checkParquetFile[T <: Product : ClassTag: TypeTag](data: Seq[T]): Unit = { - withParquetDataFrame(data)(r => checkAnswer(r, data.map(Row.fromTuple))) + withParquetDataFrame(data) { r => + r.foreach(println) + checkAnswer(r, data.map(Row.fromTuple)) + } } test("basic data types (without binary)") { @@ -148,7 +151,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } test("map") { - val data = (1 to 4).map(i => Tuple1(Map(i -> s"val_$i"))) + val data = (1 to 4).map(i => Tuple1(Map(i -> (i + 1), (i + 1) -> (i + 2)))) checkParquetFile(data) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetTest.scala index eb15a1609f1d0..a4c8350a3e670 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -42,7 +42,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { (data: Seq[T]) (f: String => Unit): Unit = { withTempPath { file => - sqlContext.createDataFrame(data).write.parquet(file.getCanonicalPath) + sqlContext.createDataFrame(data).coalesce(1).write.parquet(file.getCanonicalPath) f(file.getCanonicalPath) } } From 0525346122f00cee739aaeaa1ec70ff810f912db Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 4 Jul 2015 23:44:54 -0700 Subject: [PATCH 02/25] Removes old Parquet record converters --- .../spark/sql/parquet/ParquetConverter.scala | 831 +----------------- 1 file changed, 1 insertion(+), 830 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 86a77bf965daa..be0a2029d233b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -17,61 +17,15 @@ package org.apache.spark.sql.parquet -import java.nio.ByteOrder - -import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap} - -import org.apache.parquet.Preconditions -import org.apache.parquet.column.Dictionary -import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter} -import org.apache.parquet.schema.MessageType - import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.parquet.CatalystConverter.FieldType -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -/** - * Collection of converters of Parquet types (group and primitive types) that - * model arrays and maps. The conversions are partly based on the AvroParquet - * converters that are part of Parquet in order to be able to process these - * types. - * - * There are several types of converters: - *
    - *
  • [[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive - * (numeric, boolean and String) types
  • - *
  • [[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays - * of native JVM element types; note: currently null values are not supported!
  • - *
  • [[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of - * arbitrary element types (including nested element types); note: currently - * null values are not supported!
  • - *
  • [[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs
  • - *
  • [[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note: - * currently null values are not supported!
  • - *
  • [[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows - * of only primitive element types
  • - *
  • [[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested - * records, including the top-level row record
  • - *
- */ private[sql] object CatalystConverter { - // The type internally used for fields - type FieldType = StructField - // This is mostly Parquet convention (see, e.g., `ConversionPatterns`). // Note that "array" for the array elements is chosen by ParquetAvro. // Using a different value will result in Parquet silently dropping columns. val ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME = "bag" val ARRAY_ELEMENTS_SCHEMA_NAME = "array" - // SPARK-4520: Thrift generated parquet files have different array element - // schema names than avro. Thrift parquet uses array_schema_name + "_tuple" - // as opposed to "array" used by default. For more information, check - // TestThriftSchemaConverter.java in parquet.thrift. - val THRIFT_ARRAY_ELEMENTS_SCHEMA_NAME_SUFFIX = "_tuple" + val MAP_KEY_SCHEMA_NAME = "key" val MAP_VALUE_SCHEMA_NAME = "value" val MAP_SCHEMA_NAME = "map" @@ -80,787 +34,4 @@ private[sql] object CatalystConverter { type ArrayScalaType[T] = Seq[T] type StructScalaType[T] = InternalRow type MapScalaType[K, V] = Map[K, V] - - protected[parquet] def createConverter( - field: FieldType, - fieldIndex: Int, - parent: CatalystConverter): Converter = { - val fieldType: DataType = field.dataType - fieldType match { - case udt: UserDefinedType[_] => { - createConverter(field.copy(dataType = udt.sqlType), fieldIndex, parent) - } - // For native JVM types we use a converter with native arrays - case ArrayType(elementType: AtomicType, false) => { - new CatalystNativeArrayConverter(elementType, fieldIndex, parent) - } - // This is for other types of arrays, including those with nested fields - case ArrayType(elementType: DataType, false) => { - new CatalystArrayConverter(elementType, fieldIndex, parent) - } - case ArrayType(elementType: DataType, true) => { - new CatalystArrayContainsNullConverter(elementType, fieldIndex, parent) - } - case StructType(fields: Array[StructField]) => { - new CatalystStructConverter(fields, fieldIndex, parent) - } - case MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) => { - new CatalystMapConverter( - Array( - new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false), - new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, valueContainsNull)), - fieldIndex, - parent) - } - // Strings, Shorts and Bytes do not have a corresponding type in Parquet - // so we need to treat them separately - case StringType => - new CatalystPrimitiveStringConverter(parent, fieldIndex) - case ShortType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addInt(value: Int): Unit = - parent.updateShort(fieldIndex, value.asInstanceOf[ShortType.InternalType]) - } - } - case ByteType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addInt(value: Int): Unit = - parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.InternalType]) - } - } - case DateType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addInt(value: Int): Unit = - parent.updateDate(fieldIndex, value.asInstanceOf[DateType.InternalType]) - } - } - case d: DecimalType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addBinary(value: Binary): Unit = - parent.updateDecimal(fieldIndex, value, d) - } - } - case TimestampType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addBinary(value: Binary): Unit = - parent.updateTimestamp(fieldIndex, value) - } - } - // All other primitive types use the default converter - case ctype: DataType if ParquetTypesConverter.isPrimitiveType(ctype) => { - // note: need the type tag here! - new CatalystPrimitiveConverter(parent, fieldIndex) - } - case _ => throw new RuntimeException( - s"unable to convert datatype ${field.dataType.toString} in CatalystConverter") - } - } - - protected[parquet] def createRootConverter( - parquetSchema: MessageType, - attributes: Seq[Attribute]): CatalystConverter = { - // For non-nested types we use the optimized Row converter - if (attributes.forall(a => ParquetTypesConverter.isPrimitiveType(a.dataType))) { - new CatalystPrimitiveRowConverter(attributes.toArray) - } else { - new CatalystGroupConverter(attributes.toArray) - } - } -} - -private[parquet] abstract class CatalystConverter extends GroupConverter { - /** - * The number of fields this group has - */ - protected[parquet] val size: Int - - /** - * The index of this converter in the parent - */ - protected[parquet] val index: Int - - /** - * The parent converter - */ - protected[parquet] val parent: CatalystConverter - - /** - * Called by child converters to update their value in its parent (this). - * Note that if possible the more specific update methods below should be used - * to avoid auto-boxing of native JVM types. - * - * @param fieldIndex - * @param value - */ - protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit - - protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateDate(fieldIndex: Int, value: Int): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = - updateField(fieldIndex, value.getBytes) - - protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = - updateField(fieldIndex, UTF8String.fromBytes(value)) - - protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = - updateField(fieldIndex, readTimestamp(value)) - - protected[parquet] def updateDecimal(fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = - updateField(fieldIndex, readDecimal(new Decimal(), value, ctype)) - - protected[parquet] def isRootConverter: Boolean = parent == null - - protected[parquet] def clearBuffer(): Unit - - /** - * Should only be called in the root (group) converter! - * - * @return - */ - def getCurrentRecord: InternalRow = throw new UnsupportedOperationException - - /** - * Read a decimal value from a Parquet Binary into "dest". Only supports decimals that fit in - * a long (i.e. precision <= 18) - * - * Returned value is needed by CatalystConverter, which doesn't reuse the Decimal object. - */ - protected[parquet] def readDecimal(dest: Decimal, value: Binary, ctype: DecimalType): Decimal = { - val precision = ctype.precisionInfo.get.precision - val scale = ctype.precisionInfo.get.scale - val bytes = value.getBytes - require(bytes.length <= 16, "Decimal field too large to read") - var unscaled = 0L - var i = 0 - while (i < bytes.length) { - unscaled = (unscaled << 8) | (bytes(i) & 0xFF) - i += 1 - } - // Make sure unscaled has the right sign, by sign-extending the first bit - val numBits = 8 * bytes.length - unscaled = (unscaled << (64 - numBits)) >> (64 - numBits) - dest.set(unscaled, precision, scale) - } - - /** - * Read a Timestamp value from a Parquet Int96Value - */ - protected[parquet] def readTimestamp(value: Binary): Long = { - Preconditions.checkArgument(value.length() == 12, "Must be 12 bytes") - val buf = value.toByteBuffer - buf.order(ByteOrder.LITTLE_ENDIAN) - val timeOfDayNanos = buf.getLong - val julianDay = buf.getInt - DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) - } -} - -/** - * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record - * to a [[org.apache.spark.sql.catalyst.expressions.InternalRow]] object. - * - * @param schema The corresponding Catalyst schema in the form of a list of attributes. - */ -private[parquet] class CatalystGroupConverter( - protected[parquet] val schema: Array[FieldType], - protected[parquet] val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var current: ArrayBuffer[Any], - protected[parquet] var buffer: ArrayBuffer[InternalRow]) - extends CatalystConverter { - - def this(schema: Array[FieldType], index: Int, parent: CatalystConverter) = - this( - schema, - index, - parent, - current = null, - buffer = new ArrayBuffer[InternalRow]( - CatalystArrayConverter.INITIAL_ARRAY_SIZE)) - - /** - * This constructor is used for the root converter only! - */ - def this(attributes: Array[Attribute]) = - this(attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), 0, null) - - protected [parquet] val converters: Array[Converter] = - schema.zipWithIndex.map { - case (field, idx) => CatalystConverter.createConverter(field, idx, this) - }.toArray - - override val size = schema.size - - override def getCurrentRecord: InternalRow = { - assert(isRootConverter, "getCurrentRecord should only be called in root group converter!") - // TODO: use iterators if possible - // Note: this will ever only be called in the root converter when the record has been - // fully processed. Therefore it will be difficult to use mutable rows instead, since - // any non-root converter never would be sure when it would be safe to re-use the buffer. - new GenericInternalRow(current.toArray) - } - - override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) - - // for child converters to update upstream values - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - current.update(fieldIndex, value) - } - - override protected[parquet] def clearBuffer(): Unit = buffer.clear() - - override def start(): Unit = { - current = ArrayBuffer.fill(size)(null) - converters.foreach { converter => - if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer() - } - } - } - - override def end(): Unit = { - if (!isRootConverter) { - assert(current != null) // there should be no empty groups - buffer.append(new GenericInternalRow(current.toArray)) - parent.updateField(index, new GenericInternalRow(buffer.toArray.asInstanceOf[Array[Any]])) - } - } -} - -/** - * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record - * to a [[org.apache.spark.sql.catalyst.expressions.InternalRow]] object. Note that his - * converter is optimized for rows of primitive types (non-nested records). - */ -private[parquet] class CatalystPrimitiveRowConverter( - protected[parquet] val schema: Array[FieldType], - protected[parquet] var current: MutableRow) - extends CatalystConverter { - - // This constructor is used for the root converter only - def this(attributes: Array[Attribute]) = - this( - attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), - new SpecificMutableRow(attributes.map(_.dataType))) - - protected [parquet] val converters: Array[Converter] = - schema.zipWithIndex.map { - case (field, idx) => CatalystConverter.createConverter(field, idx, this) - }.toArray - - override val size = schema.size - - override val index = 0 - - override val parent = null - - // Should be only called in root group converter! - override def getCurrentRecord: InternalRow = current - - override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) - - // for child converters to update upstream values - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - throw new UnsupportedOperationException // child converters should use the - // specific update methods below - } - - override protected[parquet] def clearBuffer(): Unit = {} - - override def start(): Unit = { - var i = 0 - while (i < size) { - current.setNullAt(i) - i = i + 1 - } - } - - override def end(): Unit = {} - - // Overridden here to avoid auto-boxing for primitive types - override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = - current.setBoolean(fieldIndex, value) - - override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = - current.setInt(fieldIndex, value) - - override protected[parquet] def updateDate(fieldIndex: Int, value: Int): Unit = - current.setInt(fieldIndex, value) - - override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = - current.setLong(fieldIndex, value) - - override protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit = - current.setShort(fieldIndex, value) - - override protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit = - current.setByte(fieldIndex, value) - - override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = - current.setDouble(fieldIndex, value) - - override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = - current.setFloat(fieldIndex, value) - - override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = - current.update(fieldIndex, value.getBytes) - - override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = - current.update(fieldIndex, UTF8String.fromBytes(value)) - - override protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = - current.setLong(fieldIndex, readTimestamp(value)) - - override protected[parquet] def updateDecimal( - fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = { - var decimal = current(fieldIndex).asInstanceOf[Decimal] - if (decimal == null) { - decimal = new Decimal - current(fieldIndex) = decimal - } - readDecimal(decimal, value, ctype) - } -} - -/** - * A `parquet.io.api.PrimitiveConverter` that converts Parquet types to Catalyst types. - * - * @param parent The parent group converter. - * @param fieldIndex The index inside the record. - */ -private[parquet] class CatalystPrimitiveConverter( - parent: CatalystConverter, - fieldIndex: Int) extends PrimitiveConverter { - override def addBinary(value: Binary): Unit = - parent.updateBinary(fieldIndex, value) - - override def addBoolean(value: Boolean): Unit = - parent.updateBoolean(fieldIndex, value) - - override def addDouble(value: Double): Unit = - parent.updateDouble(fieldIndex, value) - - override def addFloat(value: Float): Unit = - parent.updateFloat(fieldIndex, value) - - override def addInt(value: Int): Unit = - parent.updateInt(fieldIndex, value) - - override def addLong(value: Long): Unit = - parent.updateLong(fieldIndex, value) -} - -/** - * A `parquet.io.api.PrimitiveConverter` that converts Parquet Binary to Catalyst String. - * Supports dictionaries to reduce Binary to String conversion overhead. - * - * Follows pattern in Parquet of using dictionaries, where supported, for String conversion. - * - * @param parent The parent group converter. - * @param fieldIndex The index inside the record. - */ -private[parquet] class CatalystPrimitiveStringConverter(parent: CatalystConverter, fieldIndex: Int) - extends CatalystPrimitiveConverter(parent, fieldIndex) { - - private[this] var dict: Array[Array[Byte]] = null - - override def hasDictionarySupport: Boolean = true - - override def setDictionary(dictionary: Dictionary): Unit = - dict = Array.tabulate(dictionary.getMaxId + 1) { dictionary.decodeToBinary(_).getBytes } - - override def addValueFromDictionary(dictionaryId: Int): Unit = - parent.updateString(fieldIndex, dict(dictionaryId)) - - override def addBinary(value: Binary): Unit = - parent.updateString(fieldIndex, value.getBytes) -} - -private[parquet] object CatalystArrayConverter { - val INITIAL_ARRAY_SIZE = 20 -} - -/** - * A `parquet.io.api.GroupConverter` that converts a single-element groups that - * match the characteristics of an array (see - * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.types.ArrayType]]. - * - * @param elementType The type of the array elements (complex or primitive) - * @param index The position of this (array) field inside its parent converter - * @param parent The parent converter - * @param buffer A data buffer - */ -private[parquet] class CatalystArrayConverter( - val elementType: DataType, - val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var buffer: Buffer[Any]) - extends CatalystConverter { - - def this(elementType: DataType, index: Int, parent: CatalystConverter) = - this( - elementType, - index, - parent, - new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) - - protected[parquet] val converter: Converter = CatalystConverter.createConverter( - new CatalystConverter.FieldType( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - elementType, - false), - fieldIndex = 0, - parent = this) - - override def getConverter(fieldIndex: Int): Converter = converter - - // arrays have only one (repeated) field, which is its elements - override val size = 1 - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - // fieldIndex is ignored (assumed to be zero but not checked) - if (value == null) { - throw new IllegalArgumentException("Null values inside Parquet arrays are not supported!") - } - buffer += value - } - - override protected[parquet] def clearBuffer(): Unit = { - buffer.clear() - } - - override def start(): Unit = { - if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer() - } - } - - override def end(): Unit = { - assert(parent != null) - // here we need to make sure to use ArrayScalaType - parent.updateField(index, buffer.toArray.toSeq) - clearBuffer() - } -} - -/** - * A `parquet.io.api.GroupConverter` that converts a single-element groups that - * match the characteristics of an array (see - * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.types.ArrayType]]. - * - * @param elementType The type of the array elements (native) - * @param index The position of this (array) field inside its parent converter - * @param parent The parent converter - * @param capacity The (initial) capacity of the buffer - */ -private[parquet] class CatalystNativeArrayConverter( - val elementType: AtomicType, - val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var capacity: Int = CatalystArrayConverter.INITIAL_ARRAY_SIZE) - extends CatalystConverter { - - type NativeType = elementType.InternalType - - private var buffer: Array[NativeType] = elementType.classTag.newArray(capacity) - - private var elements: Int = 0 - - protected[parquet] val converter: Converter = CatalystConverter.createConverter( - new CatalystConverter.FieldType( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - elementType, - false), - fieldIndex = 0, - parent = this) - - override def getConverter(fieldIndex: Int): Converter = converter - - // arrays have only one (repeated) field, which is its elements - override val size = 1 - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = - throw new UnsupportedOperationException - - // Overridden here to avoid auto-boxing for primitive types - override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { - checkGrowBuffer() - buffer(elements) = value.getBytes.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = { - checkGrowBuffer() - buffer(elements) = UTF8String.fromBytes(value).asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def clearBuffer(): Unit = { - elements = 0 - } - - override def start(): Unit = {} - - override def end(): Unit = { - assert(parent != null) - // here we need to make sure to use ArrayScalaType - parent.updateField( - index, - buffer.slice(0, elements).toSeq) - clearBuffer() - } - - private def checkGrowBuffer(): Unit = { - if (elements >= capacity) { - val newCapacity = 2 * capacity - val tmp: Array[NativeType] = elementType.classTag.newArray(newCapacity) - Array.copy(buffer, 0, tmp, 0, capacity) - buffer = tmp - capacity = newCapacity - } - } -} - -/** - * A `parquet.io.api.GroupConverter` that converts a single-element groups that - * match the characteristics of an array contains null (see - * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.types.ArrayType]]. - * - * @param elementType The type of the array elements (complex or primitive) - * @param index The position of this (array) field inside its parent converter - * @param parent The parent converter - * @param buffer A data buffer - */ -private[parquet] class CatalystArrayContainsNullConverter( - val elementType: DataType, - val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var buffer: Buffer[Any]) - extends CatalystConverter { - - def this(elementType: DataType, index: Int, parent: CatalystConverter) = - this( - elementType, - index, - parent, - new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) - - protected[parquet] val converter: Converter = new CatalystConverter { - - private var current: Any = null - - val converter = CatalystConverter.createConverter( - new CatalystConverter.FieldType( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - elementType, - false), - fieldIndex = 0, - parent = this) - - override def getConverter(fieldIndex: Int): Converter = converter - - override def end(): Unit = parent.updateField(index, current) - - override def start(): Unit = { - current = null - } - - override protected[parquet] val size: Int = 1 - override protected[parquet] val index: Int = 0 - override protected[parquet] val parent = CatalystArrayContainsNullConverter.this - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - current = value - } - - override protected[parquet] def clearBuffer(): Unit = {} - } - - override def getConverter(fieldIndex: Int): Converter = converter - - // arrays have only one (repeated) field, which is its elements - override val size = 1 - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - buffer += value - } - - override protected[parquet] def clearBuffer(): Unit = { - buffer.clear() - } - - override def start(): Unit = {} - - override def end(): Unit = { - assert(parent != null) - // here we need to make sure to use ArrayScalaType - parent.updateField(index, buffer.toArray.toSeq) - clearBuffer() - } -} - -/** - * This converter is for multi-element groups of primitive or complex types - * that have repetition level optional or required (so struct fields). - * - * @param schema The corresponding Catalyst schema in the form of a list of - * attributes. - * @param index - * @param parent - */ -private[parquet] class CatalystStructConverter( - override protected[parquet] val schema: Array[FieldType], - override protected[parquet] val index: Int, - override protected[parquet] val parent: CatalystConverter) - extends CatalystGroupConverter(schema, index, parent) { - - override protected[parquet] def clearBuffer(): Unit = {} - - // TODO: think about reusing the buffer - override def end(): Unit = { - assert(!isRootConverter) - // here we need to make sure to use StructScalaType - // Note: we need to actually make a copy of the array since we - // may be in a nested field - parent.updateField(index, new GenericInternalRow(current.toArray)) - } -} - -/** - * A `parquet.io.api.GroupConverter` that converts two-element groups that - * match the characteristics of a map (see - * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.types.MapType]]. - * - * @param schema - * @param index - * @param parent - */ -private[parquet] class CatalystMapConverter( - protected[parquet] val schema: Array[FieldType], - override protected[parquet] val index: Int, - override protected[parquet] val parent: CatalystConverter) - extends CatalystConverter { - - private val map = new HashMap[Any, Any]() - - private val keyValueConverter = new CatalystConverter { - private var currentKey: Any = null - private var currentValue: Any = null - val keyConverter = CatalystConverter.createConverter(schema(0), 0, this) - val valueConverter = CatalystConverter.createConverter(schema(1), 1, this) - - override def getConverter(fieldIndex: Int): Converter = { - if (fieldIndex == 0) keyConverter else valueConverter - } - - override def end(): Unit = CatalystMapConverter.this.map += currentKey -> currentValue - - override def start(): Unit = { - currentKey = null - currentValue = null - } - - override protected[parquet] val size: Int = 2 - override protected[parquet] val index: Int = 0 - override protected[parquet] val parent: CatalystConverter = CatalystMapConverter.this - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - fieldIndex match { - case 0 => - currentKey = value - case 1 => - currentValue = value - case _ => - new RuntimePermission(s"trying to update Map with fieldIndex $fieldIndex") - } - } - - override protected[parquet] def clearBuffer(): Unit = {} - } - - override protected[parquet] val size: Int = 1 - - override protected[parquet] def clearBuffer(): Unit = {} - - override def start(): Unit = { - map.clear() - } - - override def end(): Unit = { - // here we need to make sure to use MapScalaType - parent.updateField(index, map.toMap) - } - - override def getConverter(fieldIndex: Int): Converter = keyValueConverter - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = - throw new UnsupportedOperationException } From a74fb2cd8ae2cabbde52a5bd68e1afb4b222da30 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 00:30:37 -0700 Subject: [PATCH 03/25] More comments --- .../sql/parquet/CatalystRowConverter.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala index b7b0c6c782f3a..1823ed59e418f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala @@ -55,11 +55,15 @@ private[parquet] trait ParentContainerUpdater { private[parquet] object NoopUpdater extends ParentContainerUpdater /** - * This Parquet converter converts Parquet records to Spark SQL [[Row]]s. + * A [[CatalystRowConverter]] is used to convert Parquet "structs" into Spark SQL [[Row]]s. Since + * any Parquet record is also a struct, this converter can also be used as root converter. + * + * When used as a root converter, [[NoopUpdater]] should be used since root converters don't have + * any "parent" container. * * @param parquetType Parquet schema of Parquet records * @param catalystType Spark SQL schema that corresponds to the Parquet record type - * @param updater An updater which takes care of the converted row object + * @param updater An updater which propagates converted field values to the parent container */ private[parquet] class CatalystRowConverter( parquetType: GroupType, @@ -68,8 +72,8 @@ private[parquet] class CatalystRowConverter( extends GroupConverter { /** - * Updater used together with field converters of [[CatalystRowConverter]]. It sets converted - * filed values to the `ordinal`-th cell in `currentRow`. + * Updater used together with field converters within a [[CatalystRowConverter]]. It propagates + * converted filed values to the `ordinal`-th cell in `currentRow`. */ private final class RowUpdater(row: MutableRow, ordinal: Int) extends ParentContainerUpdater { override def set(value: Any): Unit = row(ordinal) = value @@ -187,9 +191,9 @@ private[parquet] class CatalystRowConverter( } /** - * Parquet converter for Parquet primitive types. Note that not all Spark SQL primitive types + * Parquet converter for Parquet primitive types. Note that not all Spark SQL atomic types * are handled by this converter. Parquet primitive types are only a subset of those of Spark - * SQL. For example, BYTE, SHORT and INT in Spark SQL are all covered by INT32 in Parquet. + * SQL. For example, BYTE, SHORT, and INT in Spark SQL are all covered by INT32 in Parquet. */ private final class CatalystPrimitiveConverter(updater: ParentContainerUpdater) extends PrimitiveConverter { @@ -311,9 +315,7 @@ private[parquet] class CatalystRowConverter( override def end(): Unit = updater.set(currentArray) - override def start(): Unit = { - currentArray = ArrayBuffer.empty[Any] - } + override def start(): Unit = currentArray = ArrayBuffer.empty[Any] // scalastyle:off /** @@ -383,9 +385,7 @@ private[parquet] class CatalystRowConverter( override def end(): Unit = updater.set(currentMap) - override def start(): Unit = { - currentMap = mutable.Map.empty[Any, Any] - } + override def start(): Unit = currentMap = mutable.Map.empty[Any, Any] /** Parquet converter for key-value pairs within the map. */ private final class KeyValueConverter( From bcac49f97bb0dfffa0d9a653f9aabddeaf17c766 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 00:30:50 -0700 Subject: [PATCH 04/25] Removes the 16-byte restriction of decimals --- .../org/apache/spark/sql/parquet/CatalystRowConverter.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala index 1823ed59e418f..3b201e71ddeaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala @@ -256,8 +256,6 @@ private[parquet] class CatalystRowConverter( val scale = decimalType.scale val bytes = value.getBytes - require(bytes.length <= 16, "Decimal field too large to read") - var unscaled = 0L var i = 0 From 6437d4b8210c1ea393a84d6897f1d1400537b716 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 02:26:39 -0700 Subject: [PATCH 05/25] Assembles requested schema from Parquet file schema --- .../spark/sql/parquet/ParquetTableSupport.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 67578dc88f48f..514575cf85fa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -90,17 +90,20 @@ private[parquet] class RowReadSupport extends ReadSupport[InternalRow] with Logg val maybeRowSchema = Option(conf.get(RowWriteSupport.SPARK_ROW_SCHEMA)) val maybeRequestedSchema = Option(conf.get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA)) - val parquetSchema = maybeRequestedSchema.map { schemaString => - val toParquet = new CatalystSchemaConverter(conf) - toParquet.convert(StructType.fromString(schemaString)) - }.getOrElse(context.getFileSchema) + val parquetRequestedSchema = + maybeRequestedSchema.map { schemaString => + StructType.fromString(schemaString).map { field => + val fieldType = context.getFileSchema.asGroupType().getType(field.name) + new MessageType("root", fieldType) + }.reduce(_ union _) + }.getOrElse(context.getFileSchema) val metadata = Map.empty[String, String] ++ maybeRequestedSchema.map(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA -> _) ++ maybeRowSchema.map(RowWriteSupport.SPARK_ROW_SCHEMA -> _) - new ReadContext(parquetSchema, metadata) + new ReadContext(parquetRequestedSchema, metadata) } } From 1781dffc058bf858d93562d642febbf038203875 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 02:27:05 -0700 Subject: [PATCH 06/25] Adds test case for SPARK-8811 --- .../apache/spark/sql/hive/parquetSuites.scala | 31 ++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index c2e09800933b5..19c8175cfef6f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -23,10 +23,12 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} import org.apache.spark.sql.hive.execution.HiveTableScan +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan} import org.apache.spark.sql.sources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, QueryTest, Row, SQLConf, SaveMode} import org.apache.spark.util.Utils @@ -685,6 +687,31 @@ class ParquetSourceSuiteBase extends ParquetPartitioningTest { sql("drop table spark_6016_fix") } + + test("SPARK-8811: compatibility with array of struct in Hive") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withTable("array_of_struct") { + val conf = Seq( + HiveContext.CONVERT_METASTORE_PARQUET.key -> "false", + SQLConf.PARQUET_BINARY_AS_STRING.key -> "true", + SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.key -> "true") + + withSQLConf(conf: _*) { + sql( + s"""CREATE TABLE array_of_struct + |STORED AS PARQUET LOCATION '$path' + |AS SELECT '1st', '2nd', ARRAY(NAMED_STRUCT('a', 'val_a', 'b', 'val_b')) + """.stripMargin) + + checkAnswer( + sqlContext.read.parquet(path), + Row("1st", "2nd", Seq(Row("val_a", "val_b")))) + } + } + } + } } class ParquetDataSourceOnSourceSuite extends ParquetSourceSuiteBase { @@ -762,7 +789,9 @@ class ParquetDataSourceOffSourceSuite extends ParquetSourceSuiteBase { /** * A collection of tests for parquet data with various forms of partitioning. */ -abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll { +abstract class ParquetPartitioningTest extends QueryTest with SQLTestUtils with BeforeAndAfterAll { + override def sqlContext = TestHive + var partitionedTableDir: File = null var normalTableDir: File = null var partitionedTableDirWithKey: File = null From 7fb21f114db955b28b38b25141aa5af60c8e0d55 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 02:29:35 -0700 Subject: [PATCH 07/25] Reverts an unnecessary debugging change --- .../test/scala/org/apache/spark/sql/parquet/ParquetTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetTest.scala index a4c8350a3e670..eb15a1609f1d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetTest.scala @@ -42,7 +42,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { (data: Seq[T]) (f: String => Unit): Unit = { withTempPath { file => - sqlContext.createDataFrame(data).coalesce(1).write.parquet(file.getCanonicalPath) + sqlContext.createDataFrame(data).write.parquet(file.getCanonicalPath) f(file.getCanonicalPath) } } From 38fe1e753713a4bda605da83e2565e3c07a2b58b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 18:53:00 -0700 Subject: [PATCH 08/25] Adds explicit return type --- .../src/main/scala/org/apache/spark/sql/types/DataType.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index b0fdbecdaa5a8..a4c2da8e05f5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -83,7 +83,7 @@ abstract class DataType extends AbstractDataType { object DataType { - private[sql] def fromString(raw: String) = { + private[sql] def fromString(raw: String): DataType = { Try(DataType.fromJson(raw)).getOrElse(DataType.fromCaseClassString(raw)) } From 802cbd75f3105a7ba2191f3423d9d7ab5dd8ca16 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 18:55:03 -0700 Subject: [PATCH 09/25] Fixes bugs related to schema merging and empty requested columns --- .../sql/parquet/ParquetTableSupport.scala | 112 ++++++++++++++---- 1 file changed, 87 insertions(+), 25 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 514575cf85fa2..e8851ddb68026 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -63,46 +63,108 @@ private[parquet] class RowReadSupport extends ReadSupport[InternalRow] with Logg log.debug(s"Preparing for read Parquet file with message type: $fileSchema") val toCatalyst = new CatalystSchemaConverter(conf) - val parquetSchema = readContext.getRequestedSchema - val catalystSchema = - Option(readContext.getReadSupportMetadata) - .map(_.toMap) - .flatMap { metadata => - metadata - // First tries to read requested schema, which may result from projections - .get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA) - // If not available, tries to read Catalyst schema from file metadata. It's only - // available if the target file is written by Spark SQL. - .orElse(metadata.get(RowReadSupport.SPARK_METADATA_KEY)) - } - .map(StructType.fromString) - .getOrElse { - logDebug("Catalyst schema not available, falling back to Parquet message type") - toCatalyst.convert(parquetSchema) - } + val parquetRequestedSchema = readContext.getRequestedSchema + + val catalystRequestedSchema = + Option(readContext.getReadSupportMetadata).map(_.toMap).flatMap { metadata => + metadata + // First tries to read requested schema, which may result from projections + .get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA) + // If not available, tries to read Catalyst schema from file metadata. It's only + // available if the target file is written by Spark SQL. + .orElse(metadata.get(RowReadSupport.SPARK_METADATA_KEY)) + }.map(StructType.fromString).getOrElse { + logDebug("Catalyst schema not available, falling back to Parquet schema") + toCatalyst.convert(parquetRequestedSchema) + } - logDebug(s"Catalyst schema used to read Parquet files: $catalystSchema") - new RowRecordMaterializer(parquetSchema, catalystSchema) + logDebug(s"Catalyst schema used to read Parquet files: $catalystRequestedSchema") + new RowRecordMaterializer(parquetRequestedSchema, catalystRequestedSchema) } override def init(context: InitContext): ReadContext = { val conf = context.getConfiguration + + // If the target file was written by Spark SQL, we should be able to find a serialized Catalyst + // schema of this file from its the metadata. val maybeRowSchema = Option(conf.get(RowWriteSupport.SPARK_ROW_SCHEMA)) + + // Optional schema of requested columns, in the form of a string serialized from a Catalyst + // `StructType` containing all requested columns. val maybeRequestedSchema = Option(conf.get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA)) + // Below we construct a Parquet schema containing all requested columns. This schema tells + // Parquet which columns to read. + // + // If `maybeRequestedSchema` is defined, we assemble an equivalent Parquet schema. Otherwise, + // we have to fallback to the full file schema which contains all columns in the file. + // Obviously this may waste IO bandwidth since it may read more columns than requested. + // + // Two things to note: + // + // 1. It's possible that some requested columns don't exist in the target Parquet file. For + // example, in the case of schema merging, the globally merged schema may contain extra + // columns gathered from other Parquet files. These columns will be simply filled with nulls + // when actually reading the target Parquet file. + // + // 2. When `maybeRequestedSchema` is available, we can't simply convert the Catalyst schema to + // Parquet schema using `CatalystSchemaConverter`, because the mapping is not unique due to + // non-standard behaviors of some Parquet libraries/tools. For example, a Parquet file + // containing a single integer array field `f1` may have the following legacy 2-level + // structure: + // + // message root { + // optional group f1 (LIST) { + // required INT32 element; + // } + // } + // + // while `CatalystSchemaConverter` may generate a standard 3-level structure: + // + // message root { + // optional group f1 (LIST) { + // repeated group list { + // required INT32 element; + // } + // } + // } + // + // Apparently, we can't use the 2nd schema to read the target Parquet file as they have + // different physical structures. val parquetRequestedSchema = - maybeRequestedSchema.map { schemaString => - StructType.fromString(schemaString).map { field => - val fieldType = context.getFileSchema.asGroupType().getType(field.name) - new MessageType("root", fieldType) - }.reduce(_ union _) - }.getOrElse(context.getFileSchema) + maybeRequestedSchema.fold(context.getFileSchema) { schemaString => + val toParquet = new CatalystSchemaConverter(conf) + val fileSchema = context.getFileSchema.asGroupType() + val fileFieldNames = fileSchema.getFields.map(_.getName).toSet + + StructType + // Deserializes the Catalyst schema of requested columns + .fromString(schemaString) + .map { field => + if (fileFieldNames.contains(field.name)) { + // If the field exists in the target Parquet file, extracts the field type from the + // full file schema and makes a single-field Parquet schema + new MessageType("root", fileSchema.getType(field.name)) + } else { + // Otherwise, just resorts to `CatalystSchemaConverter` + toParquet.convert(StructType(Array(field))) + } + } + // Merges all single-field Parquet schemas to form a complete schema for all requested + // columns. Note that it's possible that no columns are requested at all (e.g., count + // some partition column of a partitioned Parquet table). That's why `fold` is used here + // and always fallback to an empty Parquet schema. + .fold(new MessageType("root")) { + _ union _ + } + } val metadata = Map.empty[String, String] ++ maybeRequestedSchema.map(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA -> _) ++ maybeRowSchema.map(RowWriteSupport.SPARK_ROW_SCHEMA -> _) + logInfo(s"Going to read Parquet file with these requested columns: $parquetRequestedSchema") new ReadContext(parquetRequestedSchema, metadata) } } From 884d3e6c5f90131d89e3a87dde5a92b0ed90039a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 19:26:45 -0700 Subject: [PATCH 10/25] Fixes styling issue and reverts unnecessary changes --- .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 3 ++- .../org/apache/spark/sql/parquet/ParquetIOSuite.scala | 7 ++----- .../scala/org/apache/spark/sql/hive/parquetSuites.scala | 4 ++-- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index b244c470077bc..4269ad5d56737 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -29,10 +29,11 @@ import java.util.{Calendar, TimeZone} * precision. */ object DateTimeUtils { + final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L + // see http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian final val JULIAN_DAY_OF_EPOCH = 2440587 // and .5 final val SECONDS_PER_DAY = 60 * 60 * 24L - final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L final val HUNDRED_NANOS_PER_SECOND = 1000L * 1000L * 10L final val NANOS_PER_SECOND = HUNDRED_NANOS_PER_SECOND * 100 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index c5bce6e163a35..7b16eba00d6fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -71,10 +71,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { * Writes `data` to a Parquet file, reads it back and check file contents. */ protected def checkParquetFile[T <: Product : ClassTag: TypeTag](data: Seq[T]): Unit = { - withParquetDataFrame(data) { r => - r.foreach(println) - checkAnswer(r, data.map(Row.fromTuple)) - } + withParquetDataFrame(data)(r => checkAnswer(r, data.map(Row.fromTuple))) } test("basic data types (without binary)") { @@ -151,7 +148,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } test("map") { - val data = (1 to 4).map(i => Tuple1(Map(i -> (i + 1), (i + 1) -> (i + 2)))) + val data = (1 to 4).map(i => Tuple1(Map(i -> s"val_$i"))) checkParquetFile(data) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 19c8175cfef6f..9d79a4b007d66 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -21,6 +21,7 @@ import java.io.File import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql._ import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.sql.hive.test.TestHive @@ -30,7 +31,6 @@ import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan} import org.apache.spark.sql.sources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, QueryTest, Row, SQLConf, SaveMode} import org.apache.spark.util.Utils // The data where the partitioning key exists only in the directory structure. @@ -790,7 +790,7 @@ class ParquetDataSourceOffSourceSuite extends ParquetSourceSuiteBase { * A collection of tests for parquet data with various forms of partitioning. */ abstract class ParquetPartitioningTest extends QueryTest with SQLTestUtils with BeforeAndAfterAll { - override def sqlContext = TestHive + override def sqlContext: SQLContext = TestHive var partitionedTableDir: File = null var normalTableDir: File = null From 0cc1b37726f431f561842ae1645546e65401cdb0 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 21:29:57 -0700 Subject: [PATCH 11/25] Fixes MiMa checks --- project/MimaExcludes.scala | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 680b699e9e4a1..7340a985ba042 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -81,6 +81,32 @@ object MimaExcludes { "org.apache.spark.mllib.linalg.Matrix.numNonzeros"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.numActives") + ) ++ Seq( + // SPARK-6776 Implement backwards-compatibility rules in Catalyst converters + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystGroupConverter"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystStructConverter"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystPrimitiveStringConverter"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystMapConverter"), + ProblemFilters.exclude[FinalClassProblem]( + "org.apache.spark.sql.parquet.CatalystConverter"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystNativeArrayConverter"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystNativeArrayConverter$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystArrayContainsNullConverter"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystArrayConverter$"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystPrimitiveConverter"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.parquet.CatalystArrayConverter") ) case v if v.startsWith("1.4") => Seq( From a099d3eccb22adfcc523f77f9f30dc74bd83b2f4 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Jul 2015 22:50:30 -0700 Subject: [PATCH 12/25] More comments --- .../spark/sql/parquet/CatalystRowConverter.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala index 3b201e71ddeaa..0c20eb935fc12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala @@ -313,6 +313,9 @@ private[parquet] class CatalystRowConverter( override def end(): Unit = updater.set(currentArray) + // NOTE: We can't reuse the mutable Map here and must instantiate a new `ArrayBuffer` for the + // next value. `Row.copy()` only copies row cells, it doesn't do deep copy to objects stored + // in row cells. override def start(): Unit = currentArray = ArrayBuffer.empty[Any] // scalastyle:off @@ -334,8 +337,8 @@ private[parquet] class CatalystRowConverter( * @see https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules */ // scalastyle:on - private def isElementType(repeatedType: Type, elementType: DataType): Boolean = { - (repeatedType, elementType) match { + private def isElementType(parquetRepeatedType: Type, catalystElementType: DataType): Boolean = { + (parquetRepeatedType, catalystElementType) match { case (t: PrimitiveType, _) => true case (t: GroupType, _) if t.getFieldCount > 1 => true case (t: GroupType, StructType(Array(f))) if f.name == t.getFieldName(0) => true @@ -383,6 +386,9 @@ private[parquet] class CatalystRowConverter( override def end(): Unit = updater.set(currentMap) + // NOTE: We can't reuse the mutable Map here and must instantiate a new `Map` for the next + // value. `Row.copy()` only copies row cells, it doesn't do deep copy to objects stored in row + // cells. override def start(): Unit = currentMap = mutable.Map.empty[Any, Any] /** Parquet converter for key-value pairs within the map. */ From 06cfe9de612c41a20e1633513fb0b07be48bc261 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 6 Jul 2015 12:12:02 -0700 Subject: [PATCH 13/25] Adds comments about TimestampType handling --- .../sql/parquet/CatalystRowConverter.scala | 1 + .../sql/parquet/CatalystSchemaConverter.scala | 21 ++++++++++++++++--- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala index 0c20eb935fc12..e63cf32824e78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala @@ -146,6 +146,7 @@ private[parquet] class CatalystRowConverter( new CatalystStringConverter(updater) case TimestampType => + // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. new PrimitiveConverter { override def addBinary(value: Binary): Unit = { assert( 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 4ab274ec17a02..de3a72d8146c5 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 @@ -358,9 +358,24 @@ private[parquet] class CatalystSchemaConverter( case DateType => Types.primitive(INT32, repetition).as(DATE).named(field.name) - // NOTE: !! This timestamp type is not specified in Parquet format spec !! - // However, Impala and older versions of Spark SQL use INT96 to store timestamps with - // nanosecond precision (not TIME_MILLIS or TIMESTAMP_MILLIS described in the spec). + // NOTE: Spark SQL TimestampType is NOT a well defined type in Parquet format spec. + // + // As stated in PARQUET-323, Parquet `INT96` was originally introduced to represent nanosecond + // timestamp in Impala for some historical reasons, it's not recommended to be used for any + // other types and will probably be deprecated in future Parquet format spec. That's the + // reason why Parquet format spec only defines `TIMESTAMP_MILLIS` and `TIMESTAMP_MICROS` which + // are both logical types annotating `INT64`. + // + // Originally, Spark SQL uses the same nanosecond timestamp type as Impala and Hive. Starting + // from Spark 1.5.0, we resort to a timestamp type with 100 ns precision so that we can store + // a timestamp into a `Long`. This design decision is subject to change though, for example, + // we may resort to microsecond precision in the future. + // + // For Parquet, we plan to write all `TimestampType` value as `TIMESTAMP_MICROS`, but it's + // currently not implemented yet because parquet-mr 1.7.0 (the version we're currently using) + // hasn't implemented `TIMESTAMP_MICROS` yet. + // + // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. case TimestampType => Types.primitive(INT96, repetition).named(field.name) From 13b9121161b2cf71a036940ebf7129db7bc02d36 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 7 Jul 2015 02:53:29 -0700 Subject: [PATCH 14/25] Adds ParquetAvroCompatibilitySuite --- pom.xml | 21 + sql/core/pom.xml | 36 + .../sql/parquet/CatalystRowConverter.scala | 10 +- sql/core/src/test/avro/parquet-compat.avdl | 49 + sql/core/src/test/avro/parquet-compat.avpr | 87 ++ .../parquet/test/avro/CompatibilityTest.java | 19 + .../spark/sql/parquet/test/avro/Nested.java | 196 ++++ .../parquet/test/avro/ParquetAvroCompat.java | 1001 +++++++++++++++++ .../ParquetAvroCompatibilitySuite.scala | 150 +++ 9 files changed, 1567 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/test/avro/parquet-compat.avdl create mode 100644 sql/core/src/test/avro/parquet-compat.avpr create mode 100644 sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/CompatibilityTest.java create mode 100644 sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/Nested.java create mode 100644 sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/ParquetAvroCompat.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala diff --git a/pom.xml b/pom.xml index bece526a2a3e5..82f224a4e0a51 100644 --- a/pom.xml +++ b/pom.xml @@ -161,6 +161,7 @@ 2.4.4 1.1.1.7 1.1.2 + 0.9.2 false @@ -179,6 +180,8 @@ compile compile compile + test + test + + twttr-repo + Twttr Repository + http://maven.twttr.com + + true + + + false + + spark-1.4-staging From b8c12959f456f7a1df14b7f8ec38ab922ce2055a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Jul 2015 00:57:35 -0700 Subject: [PATCH 23/25] Excludes the whole parquet package from MiMa --- project/MimaExcludes.scala | 43 ++------------------------------------ 1 file changed, 2 insertions(+), 41 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7340a985ba042..0a26b5f01d3b8 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -60,53 +60,14 @@ object MimaExcludes { "org.apache.spark.ml.regression.LeastSquaresCostFun.this"), // SQL execution is considered private. excludePackage("org.apache.spark.sql.execution"), - // NanoTime and CatalystTimestampConverter is only used inside catalyst, - // not needed anymore - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.timestamp.NanoTime"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.timestamp.NanoTime$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystTimestampConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystTimestampConverter$"), - // SPARK-6777 Implements backwards compatibility rules in CatalystSchemaConverter - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetTypeInfo"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetTypeInfo$") + // Parquet support is considered private. + excludePackage("org.apache.spark.sql.parquet") ) ++ Seq( // SPARK-8479 Add numNonzeros and numActives to Matrix. ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.numNonzeros"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.numActives") - ) ++ Seq( - // SPARK-6776 Implement backwards-compatibility rules in Catalyst converters - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystGroupConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystStructConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystPrimitiveStringConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystMapConverter"), - ProblemFilters.exclude[FinalClassProblem]( - "org.apache.spark.sql.parquet.CatalystConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystNativeArrayConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystNativeArrayConverter$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystArrayContainsNullConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystArrayConverter$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystPrimitiveConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystArrayConverter") ) case v if v.startsWith("1.4") => Seq( From c6fbc0651f66b5e75498b25fb14023563babba45 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Jul 2015 00:59:16 -0700 Subject: [PATCH 24/25] Removes WIP file committed by mistake --- .../sql/parquet/ParquetHiveCompatibilitySuite.scala | 11 ----------- 1 file changed, 11 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetHiveCompatibilitySuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetHiveCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetHiveCompatibilitySuite.scala deleted file mode 100644 index 99f6d45ec5b41..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetHiveCompatibilitySuite.scala +++ /dev/null @@ -1,11 +0,0 @@ -package org.apache.spark.sql.parquet - -class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest { - override protected def beforeAll(): Unit = { - super.beforeAll() - } - - test("Read Parquet files genenrated by parquet-hive") { - - } -} From 360fe18a61538b03cac05da1c6d258e124df6feb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Jul 2015 12:07:23 -0700 Subject: [PATCH 25/25] Adds ParquetHiveCompatibilitySuite --- .../ParquetAvroCompatibilitySuite.scala | 2 +- .../parquet/ParquetCompatibilityTest.scala | 12 +-- .../ParquetThriftCompatibilitySuite.scala | 2 +- .../hive/ParquetHiveCompatibilitySuite.scala | 92 +++++++++++++++++++ 4 files changed, 98 insertions(+), 10 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala index 51da6e0c47cb7..bfa427349ff6a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.{Row, SQLContext} class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest { import ParquetCompatibilityTest._ - override def sqlContext: SQLContext = TestSQLContext + override val sqlContext: SQLContext = TestSQLContext override protected def beforeAll(): Unit = { super.beforeAll() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala index 07a1b534b6ce6..b4cdfd9e98f6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala @@ -25,13 +25,10 @@ import org.apache.parquet.hadoop.ParquetFileReader import org.apache.parquet.schema.MessageType import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.{QueryTest, SQLContext} +import org.apache.spark.sql.QueryTest import org.apache.spark.util.Utils abstract class ParquetCompatibilityTest extends QueryTest with ParquetTest with BeforeAndAfterAll { - override def sqlContext: SQLContext = TestSQLContext - protected var parquetStore: File = _ override protected def beforeAll(): Unit = { @@ -45,10 +42,9 @@ abstract class ParquetCompatibilityTest extends QueryTest with ParquetTest with def readParquetSchema(path: String): MessageType = { val fsPath = new Path(path) - val footers = - ParquetFileReader.readAllFootersInParallel( - configuration, fsPath.getFileSystem(configuration).listStatus(fsPath).toSeq, true) - + val fs = fsPath.getFileSystem(configuration) + val parquetFiles = fs.listStatus(fsPath).toSeq.filterNot(_.getPath.getName.startsWith("_")) + val footers = ParquetFileReader.readAllFootersInParallel(configuration, parquetFiles, true) footers.head.getParquetMetadata.getFileMetaData.getSchema } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetThriftCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetThriftCompatibilitySuite.scala index 5914a974881b7..d22066cabc567 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetThriftCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetThriftCompatibilitySuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.{Row, SQLContext} class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest { import ParquetCompatibilityTest._ - override def sqlContext: SQLContext = TestSQLContext + override val sqlContext: SQLContext = TestSQLContext override protected def beforeAll(): Unit = { super.beforeAll() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala new file mode 100644 index 0000000000000..bb5f1febe9ad4 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -0,0 +1,92 @@ +/* + * 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.hive + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.parquet.ParquetCompatibilityTest +import org.apache.spark.sql.{Row, SQLConf, SQLContext} + +class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest { + import ParquetCompatibilityTest.makeNullable + + override val sqlContext: SQLContext = TestHive + + override protected def beforeAll(): Unit = { + super.beforeAll() + + withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { + withTempTable("data") { + sqlContext.sql( + s"""CREATE TABLE parquet_compat( + | bool_column BOOLEAN, + | byte_column TINYINT, + | short_column SMALLINT, + | int_column INT, + | long_column BIGINT, + | float_column FLOAT, + | double_column DOUBLE, + | + | strings_column ARRAY, + | int_to_string_column MAP + |) + |STORED AS PARQUET + |LOCATION '${parquetStore.getCanonicalPath}' + """.stripMargin) + + val schema = sqlContext.table("parquet_compat").schema + val rowRDD = sqlContext.sparkContext.parallelize(makeRows).coalesce(1) + sqlContext.createDataFrame(rowRDD, schema).registerTempTable("data") + sqlContext.sql("INSERT INTO TABLE parquet_compat SELECT * FROM data") + } + } + } + + override protected def afterAll(): Unit = { + sqlContext.sql("DROP TABLE parquet_compat") + } + + test("Read Parquet file generated by parquet-hive") { + logInfo( + s"""Schema of the Parquet file written by parquet-hive: + |${readParquetSchema(parquetStore.getCanonicalPath)} + """.stripMargin) + + // Unfortunately parquet-hive doesn't add `UTF8` annotation to BINARY when writing strings. + // Have to assume all BINARY values are strings here. + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "true") { + checkAnswer(sqlContext.read.parquet(parquetStore.getCanonicalPath), makeRows) + } + } + + def makeRows: Seq[Row] = { + (0 until 10).map { i => + def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) + + Row( + nullable(i % 2 == 0: java.lang.Boolean), + nullable(i.toByte: java.lang.Byte), + nullable((i + 1).toShort: java.lang.Short), + nullable(i + 2: Integer), + nullable(i.toLong * 10: java.lang.Long), + nullable(i.toFloat + 0.1f: java.lang.Float), + nullable(i.toDouble + 0.2d: java.lang.Double), + nullable(Seq.tabulate(3)(n => s"arr_${i + n}")), + nullable(Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap)) + } + } +}