diff --git a/.rat-excludes b/.rat-excludes index 0240e81c45ea2..236c2db05367c 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -91,3 +91,5 @@ help/* html/* INDEX .lintr +gen-java.* +.*avpr diff --git a/pom.xml b/pom.xml index bece526a2a3e5..9cf2471b51304 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 @@ -1101,6 +1116,24 @@ ${parquet.version} ${parquet.deps.scope} + + org.apache.parquet + parquet-avro + ${parquet.version} + ${parquet.test.deps.scope} + + + org.apache.parquet + parquet-thrift + ${parquet.version} + ${parquet.test.deps.scope} + + + org.apache.thrift + libthrift + ${thrift.version} + ${thrift.test.deps.scope} + org.apache.flume flume-ng-core diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 680b699e9e4a1..0a26b5f01d3b8 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -60,21 +60,8 @@ 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]( 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..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 @@ -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): DataType = { + 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/pom.xml b/sql/core/pom.xml index 8fc16928adbd9..f90099f22d4bd 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -101,9 +101,45 @@ 9.3-1102-jdbc41 test + + org.apache.parquet + parquet-avro + test + + + org.apache.parquet + parquet-thrift + test + + + org.apache.thrift + libthrift + test + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + src/test/gen-java + + + + + + 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..0c3d8fdab6bd2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala @@ -0,0 +1,434 @@ +/* + * 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 + +/** + * 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 propagates converted field values to the parent container + */ +private[parquet] class CatalystRowConverter( + parquetType: GroupType, + catalystType: StructType, + updater: ParentContainerUpdater) + extends GroupConverter { + + /** + * 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 + 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 => + // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. + new PrimitiveConverter { + // Converts nanosecond timestamps stored as INT96 + 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, new ParentContainerUpdater { + override def set(value: Any): Unit = updater.set(value.asInstanceOf[Row].copy()) + }) + + 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 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. + */ + 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 { + + // Converts decimals stored as INT32 + override def addInt(value: Int): Unit = { + addLong(value: Long) + } + + // Converts decimals stored as INT64 + override def addLong(value: Long): Unit = { + updater.set(Decimal(value, decimalType.precision, decimalType.scale)) + } + + // Converts decimals stored as either FIXED_LENGTH_BYTE_ARRAY or BINARY + 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 + + 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) + + // NOTE: We can't reuse the mutable `ArrayBuffer` here and must instantiate a new buffer 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 + /** + * 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(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 + 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) + + // 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. */ + 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/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) 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 } 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..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 @@ -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,133 @@ 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 - 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)) - } + log.debug(s"Preparing for read Parquet file with message type: $fileSchema") + + val toCatalyst = new CatalystSchemaConverter(conf) + 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) } - } - // 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) + + logDebug(s"Catalyst schema used to read Parquet files: $catalystRequestedSchema") + new RowRecordMaterializer(parquetRequestedSchema, catalystRequestedSchema) } - 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 + + // 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.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 origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) - if (origAttributesStr != null) { - metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) - } + val metadata = + Map.empty[String, String] ++ + maybeRequestedSchema.map(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA -> _) ++ + maybeRowSchema.map(RowWriteSupport.SPARK_ROW_SCHEMA -> _) - new ReadSupport.ReadContext(parquetSchema, metadata) + logInfo(s"Going to read Parquet file with these requested columns: $parquetRequestedSchema") + new ReadContext(parquetRequestedSchema, 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/README.md b/sql/core/src/test/README.md new file mode 100644 index 0000000000000..3dd9861b4896d --- /dev/null +++ b/sql/core/src/test/README.md @@ -0,0 +1,33 @@ +# Notes for Parquet compatibility tests + +The following directories and files are used for Parquet compatibility tests: + +``` +. +├── README.md # This file +├── avro +│   ├── parquet-compat.avdl # Testing Avro IDL +│   └── parquet-compat.avpr # !! NO TOUCH !! Protocol file generated from parquet-compat.avdl +├── gen-java # !! NO TOUCH !! Generated Java code +├── scripts +│   └── gen-code.sh # Script used to generate Java code for Thrift and Avro +└── thrift + └── parquet-compat.thrift # Testing Thrift schema +``` + +Generated Java code are used in the following test suites: + +- `org.apache.spark.sql.parquet.ParquetAvroCompatibilitySuite` +- `org.apache.spark.sql.parquet.ParquetThriftCompatibilitySuite` + +To avoid code generation during build time, Java code generated from testing Thrift schema and Avro IDL are also checked in. + +When updating the testing Thrift schema and Avro IDL, please run `gen-code.sh` to update all the generated Java code. + +## Prerequisites + +Please ensure `avro-tools` and `thrift` are installed. You may install these two on Mac OS X via: + +```bash +$ brew install thrift avro-tools +``` diff --git a/sql/core/src/test/avro/parquet-compat.avdl b/sql/core/src/test/avro/parquet-compat.avdl new file mode 100644 index 0000000000000..24729f6143e6c --- /dev/null +++ b/sql/core/src/test/avro/parquet-compat.avdl @@ -0,0 +1,47 @@ +/* + * 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. + */ + +// This is a test protocol for testing parquet-avro compatibility. +@namespace("org.apache.spark.sql.parquet.test.avro") +protocol CompatibilityTest { + record Nested { + array nested_ints_column; + string nested_string_column; + } + + record ParquetAvroCompat { + boolean bool_column; + int int_column; + long long_column; + float float_column; + double double_column; + bytes binary_column; + string string_column; + + union { null, boolean } maybe_bool_column; + union { null, int } maybe_int_column; + union { null, long } maybe_long_column; + union { null, float } maybe_float_column; + union { null, double } maybe_double_column; + union { null, bytes } maybe_binary_column; + union { null, string } maybe_string_column; + + array strings_column; + map string_to_int_column; + map> complex_column; + } +} diff --git a/sql/core/src/test/avro/parquet-compat.avpr b/sql/core/src/test/avro/parquet-compat.avpr new file mode 100644 index 0000000000000..a83b7c990dd2e --- /dev/null +++ b/sql/core/src/test/avro/parquet-compat.avpr @@ -0,0 +1,86 @@ +{ + "protocol" : "CompatibilityTest", + "namespace" : "org.apache.spark.sql.parquet.test.avro", + "types" : [ { + "type" : "record", + "name" : "Nested", + "fields" : [ { + "name" : "nested_ints_column", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "nested_string_column", + "type" : "string" + } ] + }, { + "type" : "record", + "name" : "ParquetAvroCompat", + "fields" : [ { + "name" : "bool_column", + "type" : "boolean" + }, { + "name" : "int_column", + "type" : "int" + }, { + "name" : "long_column", + "type" : "long" + }, { + "name" : "float_column", + "type" : "float" + }, { + "name" : "double_column", + "type" : "double" + }, { + "name" : "binary_column", + "type" : "bytes" + }, { + "name" : "string_column", + "type" : "string" + }, { + "name" : "maybe_bool_column", + "type" : [ "null", "boolean" ] + }, { + "name" : "maybe_int_column", + "type" : [ "null", "int" ] + }, { + "name" : "maybe_long_column", + "type" : [ "null", "long" ] + }, { + "name" : "maybe_float_column", + "type" : [ "null", "float" ] + }, { + "name" : "maybe_double_column", + "type" : [ "null", "double" ] + }, { + "name" : "maybe_binary_column", + "type" : [ "null", "bytes" ] + }, { + "name" : "maybe_string_column", + "type" : [ "null", "string" ] + }, { + "name" : "strings_column", + "type" : { + "type" : "array", + "items" : "string" + } + }, { + "name" : "string_to_int_column", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "complex_column", + "type" : { + "type" : "map", + "values" : { + "type" : "array", + "items" : "Nested" + } + } + } ] + } ], + "messages" : { } +} \ No newline at end of file diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/CompatibilityTest.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/CompatibilityTest.java new file mode 100644 index 0000000000000..daec65a5bbe57 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/CompatibilityTest.java @@ -0,0 +1,17 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.parquet.test.avro; + +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public interface CompatibilityTest { + public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol.parse("{\"protocol\":\"CompatibilityTest\",\"namespace\":\"org.apache.spark.sql.parquet.test.avro\",\"types\":[{\"type\":\"record\",\"name\":\"Nested\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"ParquetAvroCompat\",\"fields\":[{\"name\":\"bool_column\",\"type\":\"boolean\"},{\"name\":\"int_column\",\"type\":\"int\"},{\"name\":\"long_column\",\"type\":\"long\"},{\"name\":\"float_column\",\"type\":\"float\"},{\"name\":\"double_column\",\"type\":\"double\"},{\"name\":\"binary_column\",\"type\":\"bytes\"},{\"name\":\"string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"maybe_bool_column\",\"type\":[\"null\",\"boolean\"]},{\"name\":\"maybe_int_column\",\"type\":[\"null\",\"int\"]},{\"name\":\"maybe_long_column\",\"type\":[\"null\",\"long\"]},{\"name\":\"maybe_float_column\",\"type\":[\"null\",\"float\"]},{\"name\":\"maybe_double_column\",\"type\":[\"null\",\"double\"]},{\"name\":\"maybe_binary_column\",\"type\":[\"null\",\"bytes\"]},{\"name\":\"maybe_string_column\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}]},{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"string_to_int_column\",\"type\":{\"type\":\"map\",\"values\":\"int\",\"avro.java.string\":\"String\"}},{\"name\":\"complex_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":\"Nested\"},\"avro.java.string\":\"String\"}}]}],\"messages\":{}}"); + + @SuppressWarnings("all") + public interface Callback extends CompatibilityTest { + public static final org.apache.avro.Protocol PROTOCOL = org.apache.spark.sql.parquet.test.avro.CompatibilityTest.PROTOCOL; + } +} \ No newline at end of file diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/Nested.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/Nested.java new file mode 100644 index 0000000000000..051f1ee903863 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/Nested.java @@ -0,0 +1,196 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class Nested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"Nested\",\"namespace\":\"org.apache.spark.sql.parquet.test.avro\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public java.util.List nested_ints_column; + @Deprecated public java.lang.String nested_string_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public Nested() {} + + /** + * All-args constructor. + */ + public Nested(java.util.List nested_ints_column, java.lang.String nested_string_column) { + this.nested_ints_column = nested_ints_column; + this.nested_string_column = nested_string_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return nested_ints_column; + case 1: return nested_string_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: nested_ints_column = (java.util.List)value$; break; + case 1: nested_string_column = (java.lang.String)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'nested_ints_column' field. + */ + public java.util.List getNestedIntsColumn() { + return nested_ints_column; + } + + /** + * Sets the value of the 'nested_ints_column' field. + * @param value the value to set. + */ + public void setNestedIntsColumn(java.util.List value) { + this.nested_ints_column = value; + } + + /** + * Gets the value of the 'nested_string_column' field. + */ + public java.lang.String getNestedStringColumn() { + return nested_string_column; + } + + /** + * Sets the value of the 'nested_string_column' field. + * @param value the value to set. + */ + public void setNestedStringColumn(java.lang.String value) { + this.nested_string_column = value; + } + + /** Creates a new Nested RecordBuilder */ + public static org.apache.spark.sql.parquet.test.avro.Nested.Builder newBuilder() { + return new org.apache.spark.sql.parquet.test.avro.Nested.Builder(); + } + + /** Creates a new Nested RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.parquet.test.avro.Nested.Builder newBuilder(org.apache.spark.sql.parquet.test.avro.Nested.Builder other) { + return new org.apache.spark.sql.parquet.test.avro.Nested.Builder(other); + } + + /** Creates a new Nested RecordBuilder by copying an existing Nested instance */ + public static org.apache.spark.sql.parquet.test.avro.Nested.Builder newBuilder(org.apache.spark.sql.parquet.test.avro.Nested other) { + return new org.apache.spark.sql.parquet.test.avro.Nested.Builder(other); + } + + /** + * RecordBuilder for Nested instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.util.List nested_ints_column; + private java.lang.String nested_string_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.parquet.test.avro.Nested.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.parquet.test.avro.Nested.Builder other) { + super(other); + if (isValidValue(fields()[0], other.nested_ints_column)) { + this.nested_ints_column = data().deepCopy(fields()[0].schema(), other.nested_ints_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.nested_string_column)) { + this.nested_string_column = data().deepCopy(fields()[1].schema(), other.nested_string_column); + fieldSetFlags()[1] = true; + } + } + + /** Creates a Builder by copying an existing Nested instance */ + private Builder(org.apache.spark.sql.parquet.test.avro.Nested other) { + super(org.apache.spark.sql.parquet.test.avro.Nested.SCHEMA$); + if (isValidValue(fields()[0], other.nested_ints_column)) { + this.nested_ints_column = data().deepCopy(fields()[0].schema(), other.nested_ints_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.nested_string_column)) { + this.nested_string_column = data().deepCopy(fields()[1].schema(), other.nested_string_column); + fieldSetFlags()[1] = true; + } + } + + /** Gets the value of the 'nested_ints_column' field */ + public java.util.List getNestedIntsColumn() { + return nested_ints_column; + } + + /** Sets the value of the 'nested_ints_column' field */ + public org.apache.spark.sql.parquet.test.avro.Nested.Builder setNestedIntsColumn(java.util.List value) { + validate(fields()[0], value); + this.nested_ints_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'nested_ints_column' field has been set */ + public boolean hasNestedIntsColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'nested_ints_column' field */ + public org.apache.spark.sql.parquet.test.avro.Nested.Builder clearNestedIntsColumn() { + nested_ints_column = null; + fieldSetFlags()[0] = false; + return this; + } + + /** Gets the value of the 'nested_string_column' field */ + public java.lang.String getNestedStringColumn() { + return nested_string_column; + } + + /** Sets the value of the 'nested_string_column' field */ + public org.apache.spark.sql.parquet.test.avro.Nested.Builder setNestedStringColumn(java.lang.String value) { + validate(fields()[1], value); + this.nested_string_column = value; + fieldSetFlags()[1] = true; + return this; + } + + /** Checks whether the 'nested_string_column' field has been set */ + public boolean hasNestedStringColumn() { + return fieldSetFlags()[1]; + } + + /** Clears the value of the 'nested_string_column' field */ + public org.apache.spark.sql.parquet.test.avro.Nested.Builder clearNestedStringColumn() { + nested_string_column = null; + fieldSetFlags()[1] = false; + return this; + } + + @Override + public Nested build() { + try { + Nested record = new Nested(); + record.nested_ints_column = fieldSetFlags()[0] ? this.nested_ints_column : (java.util.List) defaultValue(fields()[0]); + record.nested_string_column = fieldSetFlags()[1] ? this.nested_string_column : (java.lang.String) defaultValue(fields()[1]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/ParquetAvroCompat.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/ParquetAvroCompat.java new file mode 100644 index 0000000000000..354c9d73cca31 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/ParquetAvroCompat.java @@ -0,0 +1,1001 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class ParquetAvroCompat extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"ParquetAvroCompat\",\"namespace\":\"org.apache.spark.sql.parquet.test.avro\",\"fields\":[{\"name\":\"bool_column\",\"type\":\"boolean\"},{\"name\":\"int_column\",\"type\":\"int\"},{\"name\":\"long_column\",\"type\":\"long\"},{\"name\":\"float_column\",\"type\":\"float\"},{\"name\":\"double_column\",\"type\":\"double\"},{\"name\":\"binary_column\",\"type\":\"bytes\"},{\"name\":\"string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"maybe_bool_column\",\"type\":[\"null\",\"boolean\"]},{\"name\":\"maybe_int_column\",\"type\":[\"null\",\"int\"]},{\"name\":\"maybe_long_column\",\"type\":[\"null\",\"long\"]},{\"name\":\"maybe_float_column\",\"type\":[\"null\",\"float\"]},{\"name\":\"maybe_double_column\",\"type\":[\"null\",\"double\"]},{\"name\":\"maybe_binary_column\",\"type\":[\"null\",\"bytes\"]},{\"name\":\"maybe_string_column\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}]},{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"string_to_int_column\",\"type\":{\"type\":\"map\",\"values\":\"int\",\"avro.java.string\":\"String\"}},{\"name\":\"complex_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":{\"type\":\"record\",\"name\":\"Nested\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]}},\"avro.java.string\":\"String\"}}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public boolean bool_column; + @Deprecated public int int_column; + @Deprecated public long long_column; + @Deprecated public float float_column; + @Deprecated public double double_column; + @Deprecated public java.nio.ByteBuffer binary_column; + @Deprecated public java.lang.String string_column; + @Deprecated public java.lang.Boolean maybe_bool_column; + @Deprecated public java.lang.Integer maybe_int_column; + @Deprecated public java.lang.Long maybe_long_column; + @Deprecated public java.lang.Float maybe_float_column; + @Deprecated public java.lang.Double maybe_double_column; + @Deprecated public java.nio.ByteBuffer maybe_binary_column; + @Deprecated public java.lang.String maybe_string_column; + @Deprecated public java.util.List strings_column; + @Deprecated public java.util.Map string_to_int_column; + @Deprecated public java.util.Map> complex_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public ParquetAvroCompat() {} + + /** + * All-args constructor. + */ + public ParquetAvroCompat(java.lang.Boolean bool_column, java.lang.Integer int_column, java.lang.Long long_column, java.lang.Float float_column, java.lang.Double double_column, java.nio.ByteBuffer binary_column, java.lang.String string_column, java.lang.Boolean maybe_bool_column, java.lang.Integer maybe_int_column, java.lang.Long maybe_long_column, java.lang.Float maybe_float_column, java.lang.Double maybe_double_column, java.nio.ByteBuffer maybe_binary_column, java.lang.String maybe_string_column, java.util.List strings_column, java.util.Map string_to_int_column, java.util.Map> complex_column) { + this.bool_column = bool_column; + this.int_column = int_column; + this.long_column = long_column; + this.float_column = float_column; + this.double_column = double_column; + this.binary_column = binary_column; + this.string_column = string_column; + this.maybe_bool_column = maybe_bool_column; + this.maybe_int_column = maybe_int_column; + this.maybe_long_column = maybe_long_column; + this.maybe_float_column = maybe_float_column; + this.maybe_double_column = maybe_double_column; + this.maybe_binary_column = maybe_binary_column; + this.maybe_string_column = maybe_string_column; + this.strings_column = strings_column; + this.string_to_int_column = string_to_int_column; + this.complex_column = complex_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return bool_column; + case 1: return int_column; + case 2: return long_column; + case 3: return float_column; + case 4: return double_column; + case 5: return binary_column; + case 6: return string_column; + case 7: return maybe_bool_column; + case 8: return maybe_int_column; + case 9: return maybe_long_column; + case 10: return maybe_float_column; + case 11: return maybe_double_column; + case 12: return maybe_binary_column; + case 13: return maybe_string_column; + case 14: return strings_column; + case 15: return string_to_int_column; + case 16: return complex_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: bool_column = (java.lang.Boolean)value$; break; + case 1: int_column = (java.lang.Integer)value$; break; + case 2: long_column = (java.lang.Long)value$; break; + case 3: float_column = (java.lang.Float)value$; break; + case 4: double_column = (java.lang.Double)value$; break; + case 5: binary_column = (java.nio.ByteBuffer)value$; break; + case 6: string_column = (java.lang.String)value$; break; + case 7: maybe_bool_column = (java.lang.Boolean)value$; break; + case 8: maybe_int_column = (java.lang.Integer)value$; break; + case 9: maybe_long_column = (java.lang.Long)value$; break; + case 10: maybe_float_column = (java.lang.Float)value$; break; + case 11: maybe_double_column = (java.lang.Double)value$; break; + case 12: maybe_binary_column = (java.nio.ByteBuffer)value$; break; + case 13: maybe_string_column = (java.lang.String)value$; break; + case 14: strings_column = (java.util.List)value$; break; + case 15: string_to_int_column = (java.util.Map)value$; break; + case 16: complex_column = (java.util.Map>)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'bool_column' field. + */ + public java.lang.Boolean getBoolColumn() { + return bool_column; + } + + /** + * Sets the value of the 'bool_column' field. + * @param value the value to set. + */ + public void setBoolColumn(java.lang.Boolean value) { + this.bool_column = value; + } + + /** + * Gets the value of the 'int_column' field. + */ + public java.lang.Integer getIntColumn() { + return int_column; + } + + /** + * Sets the value of the 'int_column' field. + * @param value the value to set. + */ + public void setIntColumn(java.lang.Integer value) { + this.int_column = value; + } + + /** + * Gets the value of the 'long_column' field. + */ + public java.lang.Long getLongColumn() { + return long_column; + } + + /** + * Sets the value of the 'long_column' field. + * @param value the value to set. + */ + public void setLongColumn(java.lang.Long value) { + this.long_column = value; + } + + /** + * Gets the value of the 'float_column' field. + */ + public java.lang.Float getFloatColumn() { + return float_column; + } + + /** + * Sets the value of the 'float_column' field. + * @param value the value to set. + */ + public void setFloatColumn(java.lang.Float value) { + this.float_column = value; + } + + /** + * Gets the value of the 'double_column' field. + */ + public java.lang.Double getDoubleColumn() { + return double_column; + } + + /** + * Sets the value of the 'double_column' field. + * @param value the value to set. + */ + public void setDoubleColumn(java.lang.Double value) { + this.double_column = value; + } + + /** + * Gets the value of the 'binary_column' field. + */ + public java.nio.ByteBuffer getBinaryColumn() { + return binary_column; + } + + /** + * Sets the value of the 'binary_column' field. + * @param value the value to set. + */ + public void setBinaryColumn(java.nio.ByteBuffer value) { + this.binary_column = value; + } + + /** + * Gets the value of the 'string_column' field. + */ + public java.lang.String getStringColumn() { + return string_column; + } + + /** + * Sets the value of the 'string_column' field. + * @param value the value to set. + */ + public void setStringColumn(java.lang.String value) { + this.string_column = value; + } + + /** + * Gets the value of the 'maybe_bool_column' field. + */ + public java.lang.Boolean getMaybeBoolColumn() { + return maybe_bool_column; + } + + /** + * Sets the value of the 'maybe_bool_column' field. + * @param value the value to set. + */ + public void setMaybeBoolColumn(java.lang.Boolean value) { + this.maybe_bool_column = value; + } + + /** + * Gets the value of the 'maybe_int_column' field. + */ + public java.lang.Integer getMaybeIntColumn() { + return maybe_int_column; + } + + /** + * Sets the value of the 'maybe_int_column' field. + * @param value the value to set. + */ + public void setMaybeIntColumn(java.lang.Integer value) { + this.maybe_int_column = value; + } + + /** + * Gets the value of the 'maybe_long_column' field. + */ + public java.lang.Long getMaybeLongColumn() { + return maybe_long_column; + } + + /** + * Sets the value of the 'maybe_long_column' field. + * @param value the value to set. + */ + public void setMaybeLongColumn(java.lang.Long value) { + this.maybe_long_column = value; + } + + /** + * Gets the value of the 'maybe_float_column' field. + */ + public java.lang.Float getMaybeFloatColumn() { + return maybe_float_column; + } + + /** + * Sets the value of the 'maybe_float_column' field. + * @param value the value to set. + */ + public void setMaybeFloatColumn(java.lang.Float value) { + this.maybe_float_column = value; + } + + /** + * Gets the value of the 'maybe_double_column' field. + */ + public java.lang.Double getMaybeDoubleColumn() { + return maybe_double_column; + } + + /** + * Sets the value of the 'maybe_double_column' field. + * @param value the value to set. + */ + public void setMaybeDoubleColumn(java.lang.Double value) { + this.maybe_double_column = value; + } + + /** + * Gets the value of the 'maybe_binary_column' field. + */ + public java.nio.ByteBuffer getMaybeBinaryColumn() { + return maybe_binary_column; + } + + /** + * Sets the value of the 'maybe_binary_column' field. + * @param value the value to set. + */ + public void setMaybeBinaryColumn(java.nio.ByteBuffer value) { + this.maybe_binary_column = value; + } + + /** + * Gets the value of the 'maybe_string_column' field. + */ + public java.lang.String getMaybeStringColumn() { + return maybe_string_column; + } + + /** + * Sets the value of the 'maybe_string_column' field. + * @param value the value to set. + */ + public void setMaybeStringColumn(java.lang.String value) { + this.maybe_string_column = value; + } + + /** + * Gets the value of the 'strings_column' field. + */ + public java.util.List getStringsColumn() { + return strings_column; + } + + /** + * Sets the value of the 'strings_column' field. + * @param value the value to set. + */ + public void setStringsColumn(java.util.List value) { + this.strings_column = value; + } + + /** + * Gets the value of the 'string_to_int_column' field. + */ + public java.util.Map getStringToIntColumn() { + return string_to_int_column; + } + + /** + * Sets the value of the 'string_to_int_column' field. + * @param value the value to set. + */ + public void setStringToIntColumn(java.util.Map value) { + this.string_to_int_column = value; + } + + /** + * Gets the value of the 'complex_column' field. + */ + public java.util.Map> getComplexColumn() { + return complex_column; + } + + /** + * Sets the value of the 'complex_column' field. + * @param value the value to set. + */ + public void setComplexColumn(java.util.Map> value) { + this.complex_column = value; + } + + /** Creates a new ParquetAvroCompat RecordBuilder */ + public static org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder newBuilder() { + return new org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder(); + } + + /** Creates a new ParquetAvroCompat RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder newBuilder(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder other) { + return new org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder(other); + } + + /** Creates a new ParquetAvroCompat RecordBuilder by copying an existing ParquetAvroCompat instance */ + public static org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder newBuilder(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat other) { + return new org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder(other); + } + + /** + * RecordBuilder for ParquetAvroCompat instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean bool_column; + private int int_column; + private long long_column; + private float float_column; + private double double_column; + private java.nio.ByteBuffer binary_column; + private java.lang.String string_column; + private java.lang.Boolean maybe_bool_column; + private java.lang.Integer maybe_int_column; + private java.lang.Long maybe_long_column; + private java.lang.Float maybe_float_column; + private java.lang.Double maybe_double_column; + private java.nio.ByteBuffer maybe_binary_column; + private java.lang.String maybe_string_column; + private java.util.List strings_column; + private java.util.Map string_to_int_column; + private java.util.Map> complex_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder other) { + super(other); + if (isValidValue(fields()[0], other.bool_column)) { + this.bool_column = data().deepCopy(fields()[0].schema(), other.bool_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int_column)) { + this.int_column = data().deepCopy(fields()[1].schema(), other.int_column); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long_column)) { + this.long_column = data().deepCopy(fields()[2].schema(), other.long_column); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float_column)) { + this.float_column = data().deepCopy(fields()[3].schema(), other.float_column); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double_column)) { + this.double_column = data().deepCopy(fields()[4].schema(), other.double_column); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.binary_column)) { + this.binary_column = data().deepCopy(fields()[5].schema(), other.binary_column); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.string_column)) { + this.string_column = data().deepCopy(fields()[6].schema(), other.string_column); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.maybe_bool_column)) { + this.maybe_bool_column = data().deepCopy(fields()[7].schema(), other.maybe_bool_column); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.maybe_int_column)) { + this.maybe_int_column = data().deepCopy(fields()[8].schema(), other.maybe_int_column); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.maybe_long_column)) { + this.maybe_long_column = data().deepCopy(fields()[9].schema(), other.maybe_long_column); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.maybe_float_column)) { + this.maybe_float_column = data().deepCopy(fields()[10].schema(), other.maybe_float_column); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.maybe_double_column)) { + this.maybe_double_column = data().deepCopy(fields()[11].schema(), other.maybe_double_column); + fieldSetFlags()[11] = true; + } + if (isValidValue(fields()[12], other.maybe_binary_column)) { + this.maybe_binary_column = data().deepCopy(fields()[12].schema(), other.maybe_binary_column); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.maybe_string_column)) { + this.maybe_string_column = data().deepCopy(fields()[13].schema(), other.maybe_string_column); + fieldSetFlags()[13] = true; + } + if (isValidValue(fields()[14], other.strings_column)) { + this.strings_column = data().deepCopy(fields()[14].schema(), other.strings_column); + fieldSetFlags()[14] = true; + } + if (isValidValue(fields()[15], other.string_to_int_column)) { + this.string_to_int_column = data().deepCopy(fields()[15].schema(), other.string_to_int_column); + fieldSetFlags()[15] = true; + } + if (isValidValue(fields()[16], other.complex_column)) { + this.complex_column = data().deepCopy(fields()[16].schema(), other.complex_column); + fieldSetFlags()[16] = true; + } + } + + /** Creates a Builder by copying an existing ParquetAvroCompat instance */ + private Builder(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat other) { + super(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.SCHEMA$); + if (isValidValue(fields()[0], other.bool_column)) { + this.bool_column = data().deepCopy(fields()[0].schema(), other.bool_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int_column)) { + this.int_column = data().deepCopy(fields()[1].schema(), other.int_column); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long_column)) { + this.long_column = data().deepCopy(fields()[2].schema(), other.long_column); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float_column)) { + this.float_column = data().deepCopy(fields()[3].schema(), other.float_column); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double_column)) { + this.double_column = data().deepCopy(fields()[4].schema(), other.double_column); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.binary_column)) { + this.binary_column = data().deepCopy(fields()[5].schema(), other.binary_column); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.string_column)) { + this.string_column = data().deepCopy(fields()[6].schema(), other.string_column); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.maybe_bool_column)) { + this.maybe_bool_column = data().deepCopy(fields()[7].schema(), other.maybe_bool_column); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.maybe_int_column)) { + this.maybe_int_column = data().deepCopy(fields()[8].schema(), other.maybe_int_column); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.maybe_long_column)) { + this.maybe_long_column = data().deepCopy(fields()[9].schema(), other.maybe_long_column); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.maybe_float_column)) { + this.maybe_float_column = data().deepCopy(fields()[10].schema(), other.maybe_float_column); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.maybe_double_column)) { + this.maybe_double_column = data().deepCopy(fields()[11].schema(), other.maybe_double_column); + fieldSetFlags()[11] = true; + } + if (isValidValue(fields()[12], other.maybe_binary_column)) { + this.maybe_binary_column = data().deepCopy(fields()[12].schema(), other.maybe_binary_column); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.maybe_string_column)) { + this.maybe_string_column = data().deepCopy(fields()[13].schema(), other.maybe_string_column); + fieldSetFlags()[13] = true; + } + if (isValidValue(fields()[14], other.strings_column)) { + this.strings_column = data().deepCopy(fields()[14].schema(), other.strings_column); + fieldSetFlags()[14] = true; + } + if (isValidValue(fields()[15], other.string_to_int_column)) { + this.string_to_int_column = data().deepCopy(fields()[15].schema(), other.string_to_int_column); + fieldSetFlags()[15] = true; + } + if (isValidValue(fields()[16], other.complex_column)) { + this.complex_column = data().deepCopy(fields()[16].schema(), other.complex_column); + fieldSetFlags()[16] = true; + } + } + + /** Gets the value of the 'bool_column' field */ + public java.lang.Boolean getBoolColumn() { + return bool_column; + } + + /** Sets the value of the 'bool_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setBoolColumn(boolean value) { + validate(fields()[0], value); + this.bool_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'bool_column' field has been set */ + public boolean hasBoolColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'bool_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearBoolColumn() { + fieldSetFlags()[0] = false; + return this; + } + + /** Gets the value of the 'int_column' field */ + public java.lang.Integer getIntColumn() { + return int_column; + } + + /** Sets the value of the 'int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setIntColumn(int value) { + validate(fields()[1], value); + this.int_column = value; + fieldSetFlags()[1] = true; + return this; + } + + /** Checks whether the 'int_column' field has been set */ + public boolean hasIntColumn() { + return fieldSetFlags()[1]; + } + + /** Clears the value of the 'int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearIntColumn() { + fieldSetFlags()[1] = false; + return this; + } + + /** Gets the value of the 'long_column' field */ + public java.lang.Long getLongColumn() { + return long_column; + } + + /** Sets the value of the 'long_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setLongColumn(long value) { + validate(fields()[2], value); + this.long_column = value; + fieldSetFlags()[2] = true; + return this; + } + + /** Checks whether the 'long_column' field has been set */ + public boolean hasLongColumn() { + return fieldSetFlags()[2]; + } + + /** Clears the value of the 'long_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearLongColumn() { + fieldSetFlags()[2] = false; + return this; + } + + /** Gets the value of the 'float_column' field */ + public java.lang.Float getFloatColumn() { + return float_column; + } + + /** Sets the value of the 'float_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setFloatColumn(float value) { + validate(fields()[3], value); + this.float_column = value; + fieldSetFlags()[3] = true; + return this; + } + + /** Checks whether the 'float_column' field has been set */ + public boolean hasFloatColumn() { + return fieldSetFlags()[3]; + } + + /** Clears the value of the 'float_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearFloatColumn() { + fieldSetFlags()[3] = false; + return this; + } + + /** Gets the value of the 'double_column' field */ + public java.lang.Double getDoubleColumn() { + return double_column; + } + + /** Sets the value of the 'double_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setDoubleColumn(double value) { + validate(fields()[4], value); + this.double_column = value; + fieldSetFlags()[4] = true; + return this; + } + + /** Checks whether the 'double_column' field has been set */ + public boolean hasDoubleColumn() { + return fieldSetFlags()[4]; + } + + /** Clears the value of the 'double_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearDoubleColumn() { + fieldSetFlags()[4] = false; + return this; + } + + /** Gets the value of the 'binary_column' field */ + public java.nio.ByteBuffer getBinaryColumn() { + return binary_column; + } + + /** Sets the value of the 'binary_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setBinaryColumn(java.nio.ByteBuffer value) { + validate(fields()[5], value); + this.binary_column = value; + fieldSetFlags()[5] = true; + return this; + } + + /** Checks whether the 'binary_column' field has been set */ + public boolean hasBinaryColumn() { + return fieldSetFlags()[5]; + } + + /** Clears the value of the 'binary_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearBinaryColumn() { + binary_column = null; + fieldSetFlags()[5] = false; + return this; + } + + /** Gets the value of the 'string_column' field */ + public java.lang.String getStringColumn() { + return string_column; + } + + /** Sets the value of the 'string_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setStringColumn(java.lang.String value) { + validate(fields()[6], value); + this.string_column = value; + fieldSetFlags()[6] = true; + return this; + } + + /** Checks whether the 'string_column' field has been set */ + public boolean hasStringColumn() { + return fieldSetFlags()[6]; + } + + /** Clears the value of the 'string_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearStringColumn() { + string_column = null; + fieldSetFlags()[6] = false; + return this; + } + + /** Gets the value of the 'maybe_bool_column' field */ + public java.lang.Boolean getMaybeBoolColumn() { + return maybe_bool_column; + } + + /** Sets the value of the 'maybe_bool_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeBoolColumn(java.lang.Boolean value) { + validate(fields()[7], value); + this.maybe_bool_column = value; + fieldSetFlags()[7] = true; + return this; + } + + /** Checks whether the 'maybe_bool_column' field has been set */ + public boolean hasMaybeBoolColumn() { + return fieldSetFlags()[7]; + } + + /** Clears the value of the 'maybe_bool_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeBoolColumn() { + maybe_bool_column = null; + fieldSetFlags()[7] = false; + return this; + } + + /** Gets the value of the 'maybe_int_column' field */ + public java.lang.Integer getMaybeIntColumn() { + return maybe_int_column; + } + + /** Sets the value of the 'maybe_int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeIntColumn(java.lang.Integer value) { + validate(fields()[8], value); + this.maybe_int_column = value; + fieldSetFlags()[8] = true; + return this; + } + + /** Checks whether the 'maybe_int_column' field has been set */ + public boolean hasMaybeIntColumn() { + return fieldSetFlags()[8]; + } + + /** Clears the value of the 'maybe_int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeIntColumn() { + maybe_int_column = null; + fieldSetFlags()[8] = false; + return this; + } + + /** Gets the value of the 'maybe_long_column' field */ + public java.lang.Long getMaybeLongColumn() { + return maybe_long_column; + } + + /** Sets the value of the 'maybe_long_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeLongColumn(java.lang.Long value) { + validate(fields()[9], value); + this.maybe_long_column = value; + fieldSetFlags()[9] = true; + return this; + } + + /** Checks whether the 'maybe_long_column' field has been set */ + public boolean hasMaybeLongColumn() { + return fieldSetFlags()[9]; + } + + /** Clears the value of the 'maybe_long_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeLongColumn() { + maybe_long_column = null; + fieldSetFlags()[9] = false; + return this; + } + + /** Gets the value of the 'maybe_float_column' field */ + public java.lang.Float getMaybeFloatColumn() { + return maybe_float_column; + } + + /** Sets the value of the 'maybe_float_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeFloatColumn(java.lang.Float value) { + validate(fields()[10], value); + this.maybe_float_column = value; + fieldSetFlags()[10] = true; + return this; + } + + /** Checks whether the 'maybe_float_column' field has been set */ + public boolean hasMaybeFloatColumn() { + return fieldSetFlags()[10]; + } + + /** Clears the value of the 'maybe_float_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeFloatColumn() { + maybe_float_column = null; + fieldSetFlags()[10] = false; + return this; + } + + /** Gets the value of the 'maybe_double_column' field */ + public java.lang.Double getMaybeDoubleColumn() { + return maybe_double_column; + } + + /** Sets the value of the 'maybe_double_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeDoubleColumn(java.lang.Double value) { + validate(fields()[11], value); + this.maybe_double_column = value; + fieldSetFlags()[11] = true; + return this; + } + + /** Checks whether the 'maybe_double_column' field has been set */ + public boolean hasMaybeDoubleColumn() { + return fieldSetFlags()[11]; + } + + /** Clears the value of the 'maybe_double_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeDoubleColumn() { + maybe_double_column = null; + fieldSetFlags()[11] = false; + return this; + } + + /** Gets the value of the 'maybe_binary_column' field */ + public java.nio.ByteBuffer getMaybeBinaryColumn() { + return maybe_binary_column; + } + + /** Sets the value of the 'maybe_binary_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeBinaryColumn(java.nio.ByteBuffer value) { + validate(fields()[12], value); + this.maybe_binary_column = value; + fieldSetFlags()[12] = true; + return this; + } + + /** Checks whether the 'maybe_binary_column' field has been set */ + public boolean hasMaybeBinaryColumn() { + return fieldSetFlags()[12]; + } + + /** Clears the value of the 'maybe_binary_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeBinaryColumn() { + maybe_binary_column = null; + fieldSetFlags()[12] = false; + return this; + } + + /** Gets the value of the 'maybe_string_column' field */ + public java.lang.String getMaybeStringColumn() { + return maybe_string_column; + } + + /** Sets the value of the 'maybe_string_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeStringColumn(java.lang.String value) { + validate(fields()[13], value); + this.maybe_string_column = value; + fieldSetFlags()[13] = true; + return this; + } + + /** Checks whether the 'maybe_string_column' field has been set */ + public boolean hasMaybeStringColumn() { + return fieldSetFlags()[13]; + } + + /** Clears the value of the 'maybe_string_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeStringColumn() { + maybe_string_column = null; + fieldSetFlags()[13] = false; + return this; + } + + /** Gets the value of the 'strings_column' field */ + public java.util.List getStringsColumn() { + return strings_column; + } + + /** Sets the value of the 'strings_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setStringsColumn(java.util.List value) { + validate(fields()[14], value); + this.strings_column = value; + fieldSetFlags()[14] = true; + return this; + } + + /** Checks whether the 'strings_column' field has been set */ + public boolean hasStringsColumn() { + return fieldSetFlags()[14]; + } + + /** Clears the value of the 'strings_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearStringsColumn() { + strings_column = null; + fieldSetFlags()[14] = false; + return this; + } + + /** Gets the value of the 'string_to_int_column' field */ + public java.util.Map getStringToIntColumn() { + return string_to_int_column; + } + + /** Sets the value of the 'string_to_int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setStringToIntColumn(java.util.Map value) { + validate(fields()[15], value); + this.string_to_int_column = value; + fieldSetFlags()[15] = true; + return this; + } + + /** Checks whether the 'string_to_int_column' field has been set */ + public boolean hasStringToIntColumn() { + return fieldSetFlags()[15]; + } + + /** Clears the value of the 'string_to_int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearStringToIntColumn() { + string_to_int_column = null; + fieldSetFlags()[15] = false; + return this; + } + + /** Gets the value of the 'complex_column' field */ + public java.util.Map> getComplexColumn() { + return complex_column; + } + + /** Sets the value of the 'complex_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setComplexColumn(java.util.Map> value) { + validate(fields()[16], value); + this.complex_column = value; + fieldSetFlags()[16] = true; + return this; + } + + /** Checks whether the 'complex_column' field has been set */ + public boolean hasComplexColumn() { + return fieldSetFlags()[16]; + } + + /** Clears the value of the 'complex_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearComplexColumn() { + complex_column = null; + fieldSetFlags()[16] = false; + return this; + } + + @Override + public ParquetAvroCompat build() { + try { + ParquetAvroCompat record = new ParquetAvroCompat(); + record.bool_column = fieldSetFlags()[0] ? this.bool_column : (java.lang.Boolean) defaultValue(fields()[0]); + record.int_column = fieldSetFlags()[1] ? this.int_column : (java.lang.Integer) defaultValue(fields()[1]); + record.long_column = fieldSetFlags()[2] ? this.long_column : (java.lang.Long) defaultValue(fields()[2]); + record.float_column = fieldSetFlags()[3] ? this.float_column : (java.lang.Float) defaultValue(fields()[3]); + record.double_column = fieldSetFlags()[4] ? this.double_column : (java.lang.Double) defaultValue(fields()[4]); + record.binary_column = fieldSetFlags()[5] ? this.binary_column : (java.nio.ByteBuffer) defaultValue(fields()[5]); + record.string_column = fieldSetFlags()[6] ? this.string_column : (java.lang.String) defaultValue(fields()[6]); + record.maybe_bool_column = fieldSetFlags()[7] ? this.maybe_bool_column : (java.lang.Boolean) defaultValue(fields()[7]); + record.maybe_int_column = fieldSetFlags()[8] ? this.maybe_int_column : (java.lang.Integer) defaultValue(fields()[8]); + record.maybe_long_column = fieldSetFlags()[9] ? this.maybe_long_column : (java.lang.Long) defaultValue(fields()[9]); + record.maybe_float_column = fieldSetFlags()[10] ? this.maybe_float_column : (java.lang.Float) defaultValue(fields()[10]); + record.maybe_double_column = fieldSetFlags()[11] ? this.maybe_double_column : (java.lang.Double) defaultValue(fields()[11]); + record.maybe_binary_column = fieldSetFlags()[12] ? this.maybe_binary_column : (java.nio.ByteBuffer) defaultValue(fields()[12]); + record.maybe_string_column = fieldSetFlags()[13] ? this.maybe_string_column : (java.lang.String) defaultValue(fields()[13]); + record.strings_column = fieldSetFlags()[14] ? this.strings_column : (java.util.List) defaultValue(fields()[14]); + record.string_to_int_column = fieldSetFlags()[15] ? this.string_to_int_column : (java.util.Map) defaultValue(fields()[15]); + record.complex_column = fieldSetFlags()[16] ? this.complex_column : (java.util.Map>) defaultValue(fields()[16]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Nested.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Nested.java new file mode 100644 index 0000000000000..281e60cc3ae34 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Nested.java @@ -0,0 +1,541 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.spark.sql.parquet.test.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-7") +public class Nested implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Nested"); + + private static final org.apache.thrift.protocol.TField NESTED_INTS_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("nestedIntsColumn", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NESTED_STRING_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("nestedStringColumn", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NestedStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NestedTupleSchemeFactory()); + } + + public List nestedIntsColumn; // required + public String nestedStringColumn; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NESTED_INTS_COLUMN((short)1, "nestedIntsColumn"), + NESTED_STRING_COLUMN((short)2, "nestedStringColumn"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NESTED_INTS_COLUMN + return NESTED_INTS_COLUMN; + case 2: // NESTED_STRING_COLUMN + return NESTED_STRING_COLUMN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NESTED_INTS_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("nestedIntsColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + tmpMap.put(_Fields.NESTED_STRING_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("nestedStringColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Nested.class, metaDataMap); + } + + public Nested() { + } + + public Nested( + List nestedIntsColumn, + String nestedStringColumn) + { + this(); + this.nestedIntsColumn = nestedIntsColumn; + this.nestedStringColumn = nestedStringColumn; + } + + /** + * Performs a deep copy on other. + */ + public Nested(Nested other) { + if (other.isSetNestedIntsColumn()) { + List __this__nestedIntsColumn = new ArrayList(other.nestedIntsColumn); + this.nestedIntsColumn = __this__nestedIntsColumn; + } + if (other.isSetNestedStringColumn()) { + this.nestedStringColumn = other.nestedStringColumn; + } + } + + public Nested deepCopy() { + return new Nested(this); + } + + @Override + public void clear() { + this.nestedIntsColumn = null; + this.nestedStringColumn = null; + } + + public int getNestedIntsColumnSize() { + return (this.nestedIntsColumn == null) ? 0 : this.nestedIntsColumn.size(); + } + + public java.util.Iterator getNestedIntsColumnIterator() { + return (this.nestedIntsColumn == null) ? null : this.nestedIntsColumn.iterator(); + } + + public void addToNestedIntsColumn(int elem) { + if (this.nestedIntsColumn == null) { + this.nestedIntsColumn = new ArrayList(); + } + this.nestedIntsColumn.add(elem); + } + + public List getNestedIntsColumn() { + return this.nestedIntsColumn; + } + + public Nested setNestedIntsColumn(List nestedIntsColumn) { + this.nestedIntsColumn = nestedIntsColumn; + return this; + } + + public void unsetNestedIntsColumn() { + this.nestedIntsColumn = null; + } + + /** Returns true if field nestedIntsColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetNestedIntsColumn() { + return this.nestedIntsColumn != null; + } + + public void setNestedIntsColumnIsSet(boolean value) { + if (!value) { + this.nestedIntsColumn = null; + } + } + + public String getNestedStringColumn() { + return this.nestedStringColumn; + } + + public Nested setNestedStringColumn(String nestedStringColumn) { + this.nestedStringColumn = nestedStringColumn; + return this; + } + + public void unsetNestedStringColumn() { + this.nestedStringColumn = null; + } + + /** Returns true if field nestedStringColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetNestedStringColumn() { + return this.nestedStringColumn != null; + } + + public void setNestedStringColumnIsSet(boolean value) { + if (!value) { + this.nestedStringColumn = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NESTED_INTS_COLUMN: + if (value == null) { + unsetNestedIntsColumn(); + } else { + setNestedIntsColumn((List)value); + } + break; + + case NESTED_STRING_COLUMN: + if (value == null) { + unsetNestedStringColumn(); + } else { + setNestedStringColumn((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NESTED_INTS_COLUMN: + return getNestedIntsColumn(); + + case NESTED_STRING_COLUMN: + return getNestedStringColumn(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NESTED_INTS_COLUMN: + return isSetNestedIntsColumn(); + case NESTED_STRING_COLUMN: + return isSetNestedStringColumn(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof Nested) + return this.equals((Nested)that); + return false; + } + + public boolean equals(Nested that) { + if (that == null) + return false; + + boolean this_present_nestedIntsColumn = true && this.isSetNestedIntsColumn(); + boolean that_present_nestedIntsColumn = true && that.isSetNestedIntsColumn(); + if (this_present_nestedIntsColumn || that_present_nestedIntsColumn) { + if (!(this_present_nestedIntsColumn && that_present_nestedIntsColumn)) + return false; + if (!this.nestedIntsColumn.equals(that.nestedIntsColumn)) + return false; + } + + boolean this_present_nestedStringColumn = true && this.isSetNestedStringColumn(); + boolean that_present_nestedStringColumn = true && that.isSetNestedStringColumn(); + if (this_present_nestedStringColumn || that_present_nestedStringColumn) { + if (!(this_present_nestedStringColumn && that_present_nestedStringColumn)) + return false; + if (!this.nestedStringColumn.equals(that.nestedStringColumn)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_nestedIntsColumn = true && (isSetNestedIntsColumn()); + list.add(present_nestedIntsColumn); + if (present_nestedIntsColumn) + list.add(nestedIntsColumn); + + boolean present_nestedStringColumn = true && (isSetNestedStringColumn()); + list.add(present_nestedStringColumn); + if (present_nestedStringColumn) + list.add(nestedStringColumn); + + return list.hashCode(); + } + + @Override + public int compareTo(Nested other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetNestedIntsColumn()).compareTo(other.isSetNestedIntsColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNestedIntsColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nestedIntsColumn, other.nestedIntsColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNestedStringColumn()).compareTo(other.isSetNestedStringColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNestedStringColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nestedStringColumn, other.nestedStringColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("Nested("); + boolean first = true; + + sb.append("nestedIntsColumn:"); + if (this.nestedIntsColumn == null) { + sb.append("null"); + } else { + sb.append(this.nestedIntsColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("nestedStringColumn:"); + if (this.nestedStringColumn == null) { + sb.append("null"); + } else { + sb.append(this.nestedStringColumn); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (nestedIntsColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nestedIntsColumn' was not present! Struct: " + toString()); + } + if (nestedStringColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nestedStringColumn' was not present! Struct: " + toString()); + } + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NestedStandardSchemeFactory implements SchemeFactory { + public NestedStandardScheme getScheme() { + return new NestedStandardScheme(); + } + } + + private static class NestedStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, Nested struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NESTED_INTS_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list0 = iprot.readListBegin(); + struct.nestedIntsColumn = new ArrayList(_list0.size); + int _elem1; + for (int _i2 = 0; _i2 < _list0.size; ++_i2) + { + _elem1 = iprot.readI32(); + struct.nestedIntsColumn.add(_elem1); + } + iprot.readListEnd(); + } + struct.setNestedIntsColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NESTED_STRING_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nestedStringColumn = iprot.readString(); + struct.setNestedStringColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, Nested struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.nestedIntsColumn != null) { + oprot.writeFieldBegin(NESTED_INTS_COLUMN_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.nestedIntsColumn.size())); + for (int _iter3 : struct.nestedIntsColumn) + { + oprot.writeI32(_iter3); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nestedStringColumn != null) { + oprot.writeFieldBegin(NESTED_STRING_COLUMN_FIELD_DESC); + oprot.writeString(struct.nestedStringColumn); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NestedTupleSchemeFactory implements SchemeFactory { + public NestedTupleScheme getScheme() { + return new NestedTupleScheme(); + } + } + + private static class NestedTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, Nested struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.nestedIntsColumn.size()); + for (int _iter4 : struct.nestedIntsColumn) + { + oprot.writeI32(_iter4); + } + } + oprot.writeString(struct.nestedStringColumn); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, Nested struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list5 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.nestedIntsColumn = new ArrayList(_list5.size); + int _elem6; + for (int _i7 = 0; _i7 < _list5.size; ++_i7) + { + _elem6 = iprot.readI32(); + struct.nestedIntsColumn.add(_elem6); + } + } + struct.setNestedIntsColumnIsSet(true); + struct.nestedStringColumn = iprot.readString(); + struct.setNestedStringColumnIsSet(true); + } + } + +} + diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/ParquetThriftCompat.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/ParquetThriftCompat.java new file mode 100644 index 0000000000000..326ae9dbaa0d1 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/ParquetThriftCompat.java @@ -0,0 +1,2808 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.spark.sql.parquet.test.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +/** + * This is a test struct for testing parquet-thrift compatibility. + */ +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-7") +public class ParquetThriftCompat implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ParquetThriftCompat"); + + private static final org.apache.thrift.protocol.TField BOOL_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("boolColumn", org.apache.thrift.protocol.TType.BOOL, (short)1); + private static final org.apache.thrift.protocol.TField BYTE_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("byteColumn", org.apache.thrift.protocol.TType.BYTE, (short)2); + private static final org.apache.thrift.protocol.TField SHORT_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("shortColumn", org.apache.thrift.protocol.TType.I16, (short)3); + private static final org.apache.thrift.protocol.TField INT_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("intColumn", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField LONG_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("longColumn", org.apache.thrift.protocol.TType.I64, (short)5); + private static final org.apache.thrift.protocol.TField DOUBLE_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleColumn", org.apache.thrift.protocol.TType.DOUBLE, (short)6); + private static final org.apache.thrift.protocol.TField BINARY_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryColumn", org.apache.thrift.protocol.TType.STRING, (short)7); + private static final org.apache.thrift.protocol.TField STRING_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("stringColumn", org.apache.thrift.protocol.TType.STRING, (short)8); + private static final org.apache.thrift.protocol.TField ENUM_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("enumColumn", org.apache.thrift.protocol.TType.I32, (short)9); + private static final org.apache.thrift.protocol.TField MAYBE_BOOL_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeBoolColumn", org.apache.thrift.protocol.TType.BOOL, (short)10); + private static final org.apache.thrift.protocol.TField MAYBE_BYTE_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeByteColumn", org.apache.thrift.protocol.TType.BYTE, (short)11); + private static final org.apache.thrift.protocol.TField MAYBE_SHORT_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeShortColumn", org.apache.thrift.protocol.TType.I16, (short)12); + private static final org.apache.thrift.protocol.TField MAYBE_INT_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeIntColumn", org.apache.thrift.protocol.TType.I32, (short)13); + private static final org.apache.thrift.protocol.TField MAYBE_LONG_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeLongColumn", org.apache.thrift.protocol.TType.I64, (short)14); + private static final org.apache.thrift.protocol.TField MAYBE_DOUBLE_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeDoubleColumn", org.apache.thrift.protocol.TType.DOUBLE, (short)15); + private static final org.apache.thrift.protocol.TField MAYBE_BINARY_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeBinaryColumn", org.apache.thrift.protocol.TType.STRING, (short)16); + private static final org.apache.thrift.protocol.TField MAYBE_STRING_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeStringColumn", org.apache.thrift.protocol.TType.STRING, (short)17); + private static final org.apache.thrift.protocol.TField MAYBE_ENUM_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeEnumColumn", org.apache.thrift.protocol.TType.I32, (short)18); + private static final org.apache.thrift.protocol.TField STRINGS_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("stringsColumn", org.apache.thrift.protocol.TType.LIST, (short)19); + private static final org.apache.thrift.protocol.TField INT_SET_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("intSetColumn", org.apache.thrift.protocol.TType.SET, (short)20); + private static final org.apache.thrift.protocol.TField INT_TO_STRING_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("intToStringColumn", org.apache.thrift.protocol.TType.MAP, (short)21); + private static final org.apache.thrift.protocol.TField COMPLEX_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("complexColumn", org.apache.thrift.protocol.TType.MAP, (short)22); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ParquetThriftCompatStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ParquetThriftCompatTupleSchemeFactory()); + } + + public boolean boolColumn; // required + public byte byteColumn; // required + public short shortColumn; // required + public int intColumn; // required + public long longColumn; // required + public double doubleColumn; // required + public ByteBuffer binaryColumn; // required + public String stringColumn; // required + /** + * + * @see Suit + */ + public Suit enumColumn; // required + public boolean maybeBoolColumn; // optional + public byte maybeByteColumn; // optional + public short maybeShortColumn; // optional + public int maybeIntColumn; // optional + public long maybeLongColumn; // optional + public double maybeDoubleColumn; // optional + public ByteBuffer maybeBinaryColumn; // optional + public String maybeStringColumn; // optional + /** + * + * @see Suit + */ + public Suit maybeEnumColumn; // optional + public List stringsColumn; // required + public Set intSetColumn; // required + public Map intToStringColumn; // required + public Map> complexColumn; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BOOL_COLUMN((short)1, "boolColumn"), + BYTE_COLUMN((short)2, "byteColumn"), + SHORT_COLUMN((short)3, "shortColumn"), + INT_COLUMN((short)4, "intColumn"), + LONG_COLUMN((short)5, "longColumn"), + DOUBLE_COLUMN((short)6, "doubleColumn"), + BINARY_COLUMN((short)7, "binaryColumn"), + STRING_COLUMN((short)8, "stringColumn"), + /** + * + * @see Suit + */ + ENUM_COLUMN((short)9, "enumColumn"), + MAYBE_BOOL_COLUMN((short)10, "maybeBoolColumn"), + MAYBE_BYTE_COLUMN((short)11, "maybeByteColumn"), + MAYBE_SHORT_COLUMN((short)12, "maybeShortColumn"), + MAYBE_INT_COLUMN((short)13, "maybeIntColumn"), + MAYBE_LONG_COLUMN((short)14, "maybeLongColumn"), + MAYBE_DOUBLE_COLUMN((short)15, "maybeDoubleColumn"), + MAYBE_BINARY_COLUMN((short)16, "maybeBinaryColumn"), + MAYBE_STRING_COLUMN((short)17, "maybeStringColumn"), + /** + * + * @see Suit + */ + MAYBE_ENUM_COLUMN((short)18, "maybeEnumColumn"), + STRINGS_COLUMN((short)19, "stringsColumn"), + INT_SET_COLUMN((short)20, "intSetColumn"), + INT_TO_STRING_COLUMN((short)21, "intToStringColumn"), + COMPLEX_COLUMN((short)22, "complexColumn"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BOOL_COLUMN + return BOOL_COLUMN; + case 2: // BYTE_COLUMN + return BYTE_COLUMN; + case 3: // SHORT_COLUMN + return SHORT_COLUMN; + case 4: // INT_COLUMN + return INT_COLUMN; + case 5: // LONG_COLUMN + return LONG_COLUMN; + case 6: // DOUBLE_COLUMN + return DOUBLE_COLUMN; + case 7: // BINARY_COLUMN + return BINARY_COLUMN; + case 8: // STRING_COLUMN + return STRING_COLUMN; + case 9: // ENUM_COLUMN + return ENUM_COLUMN; + case 10: // MAYBE_BOOL_COLUMN + return MAYBE_BOOL_COLUMN; + case 11: // MAYBE_BYTE_COLUMN + return MAYBE_BYTE_COLUMN; + case 12: // MAYBE_SHORT_COLUMN + return MAYBE_SHORT_COLUMN; + case 13: // MAYBE_INT_COLUMN + return MAYBE_INT_COLUMN; + case 14: // MAYBE_LONG_COLUMN + return MAYBE_LONG_COLUMN; + case 15: // MAYBE_DOUBLE_COLUMN + return MAYBE_DOUBLE_COLUMN; + case 16: // MAYBE_BINARY_COLUMN + return MAYBE_BINARY_COLUMN; + case 17: // MAYBE_STRING_COLUMN + return MAYBE_STRING_COLUMN; + case 18: // MAYBE_ENUM_COLUMN + return MAYBE_ENUM_COLUMN; + case 19: // STRINGS_COLUMN + return STRINGS_COLUMN; + case 20: // INT_SET_COLUMN + return INT_SET_COLUMN; + case 21: // INT_TO_STRING_COLUMN + return INT_TO_STRING_COLUMN; + case 22: // COMPLEX_COLUMN + return COMPLEX_COLUMN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __BOOLCOLUMN_ISSET_ID = 0; + private static final int __BYTECOLUMN_ISSET_ID = 1; + private static final int __SHORTCOLUMN_ISSET_ID = 2; + private static final int __INTCOLUMN_ISSET_ID = 3; + private static final int __LONGCOLUMN_ISSET_ID = 4; + private static final int __DOUBLECOLUMN_ISSET_ID = 5; + private static final int __MAYBEBOOLCOLUMN_ISSET_ID = 6; + private static final int __MAYBEBYTECOLUMN_ISSET_ID = 7; + private static final int __MAYBESHORTCOLUMN_ISSET_ID = 8; + private static final int __MAYBEINTCOLUMN_ISSET_ID = 9; + private static final int __MAYBELONGCOLUMN_ISSET_ID = 10; + private static final int __MAYBEDOUBLECOLUMN_ISSET_ID = 11; + private short __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.MAYBE_BOOL_COLUMN,_Fields.MAYBE_BYTE_COLUMN,_Fields.MAYBE_SHORT_COLUMN,_Fields.MAYBE_INT_COLUMN,_Fields.MAYBE_LONG_COLUMN,_Fields.MAYBE_DOUBLE_COLUMN,_Fields.MAYBE_BINARY_COLUMN,_Fields.MAYBE_STRING_COLUMN,_Fields.MAYBE_ENUM_COLUMN}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BOOL_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("boolColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.BYTE_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("byteColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); + tmpMap.put(_Fields.SHORT_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("shortColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + tmpMap.put(_Fields.INT_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("intColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.LONG_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("longColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.DOUBLE_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("doubleColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + tmpMap.put(_Fields.BINARY_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("binaryColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.STRING_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("stringColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ENUM_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("enumColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, Suit.class))); + tmpMap.put(_Fields.MAYBE_BOOL_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeBoolColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.MAYBE_BYTE_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeByteColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); + tmpMap.put(_Fields.MAYBE_SHORT_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeShortColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + tmpMap.put(_Fields.MAYBE_INT_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeIntColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.MAYBE_LONG_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeLongColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.MAYBE_DOUBLE_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeDoubleColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + tmpMap.put(_Fields.MAYBE_BINARY_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeBinaryColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.MAYBE_STRING_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeStringColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.MAYBE_ENUM_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeEnumColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, Suit.class))); + tmpMap.put(_Fields.STRINGS_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("stringsColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.INT_SET_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("intSetColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + tmpMap.put(_Fields.INT_TO_STRING_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("intToStringColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.COMPLEX_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("complexColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32), + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Nested.class))))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ParquetThriftCompat.class, metaDataMap); + } + + public ParquetThriftCompat() { + } + + public ParquetThriftCompat( + boolean boolColumn, + byte byteColumn, + short shortColumn, + int intColumn, + long longColumn, + double doubleColumn, + ByteBuffer binaryColumn, + String stringColumn, + Suit enumColumn, + List stringsColumn, + Set intSetColumn, + Map intToStringColumn, + Map> complexColumn) + { + this(); + this.boolColumn = boolColumn; + setBoolColumnIsSet(true); + this.byteColumn = byteColumn; + setByteColumnIsSet(true); + this.shortColumn = shortColumn; + setShortColumnIsSet(true); + this.intColumn = intColumn; + setIntColumnIsSet(true); + this.longColumn = longColumn; + setLongColumnIsSet(true); + this.doubleColumn = doubleColumn; + setDoubleColumnIsSet(true); + this.binaryColumn = org.apache.thrift.TBaseHelper.copyBinary(binaryColumn); + this.stringColumn = stringColumn; + this.enumColumn = enumColumn; + this.stringsColumn = stringsColumn; + this.intSetColumn = intSetColumn; + this.intToStringColumn = intToStringColumn; + this.complexColumn = complexColumn; + } + + /** + * Performs a deep copy on other. + */ + public ParquetThriftCompat(ParquetThriftCompat other) { + __isset_bitfield = other.__isset_bitfield; + this.boolColumn = other.boolColumn; + this.byteColumn = other.byteColumn; + this.shortColumn = other.shortColumn; + this.intColumn = other.intColumn; + this.longColumn = other.longColumn; + this.doubleColumn = other.doubleColumn; + if (other.isSetBinaryColumn()) { + this.binaryColumn = org.apache.thrift.TBaseHelper.copyBinary(other.binaryColumn); + } + if (other.isSetStringColumn()) { + this.stringColumn = other.stringColumn; + } + if (other.isSetEnumColumn()) { + this.enumColumn = other.enumColumn; + } + this.maybeBoolColumn = other.maybeBoolColumn; + this.maybeByteColumn = other.maybeByteColumn; + this.maybeShortColumn = other.maybeShortColumn; + this.maybeIntColumn = other.maybeIntColumn; + this.maybeLongColumn = other.maybeLongColumn; + this.maybeDoubleColumn = other.maybeDoubleColumn; + if (other.isSetMaybeBinaryColumn()) { + this.maybeBinaryColumn = org.apache.thrift.TBaseHelper.copyBinary(other.maybeBinaryColumn); + } + if (other.isSetMaybeStringColumn()) { + this.maybeStringColumn = other.maybeStringColumn; + } + if (other.isSetMaybeEnumColumn()) { + this.maybeEnumColumn = other.maybeEnumColumn; + } + if (other.isSetStringsColumn()) { + List __this__stringsColumn = new ArrayList(other.stringsColumn); + this.stringsColumn = __this__stringsColumn; + } + if (other.isSetIntSetColumn()) { + Set __this__intSetColumn = new HashSet(other.intSetColumn); + this.intSetColumn = __this__intSetColumn; + } + if (other.isSetIntToStringColumn()) { + Map __this__intToStringColumn = new HashMap(other.intToStringColumn); + this.intToStringColumn = __this__intToStringColumn; + } + if (other.isSetComplexColumn()) { + Map> __this__complexColumn = new HashMap>(other.complexColumn.size()); + for (Map.Entry> other_element : other.complexColumn.entrySet()) { + + Integer other_element_key = other_element.getKey(); + List other_element_value = other_element.getValue(); + + Integer __this__complexColumn_copy_key = other_element_key; + + List __this__complexColumn_copy_value = new ArrayList(other_element_value.size()); + for (Nested other_element_value_element : other_element_value) { + __this__complexColumn_copy_value.add(new Nested(other_element_value_element)); + } + + __this__complexColumn.put(__this__complexColumn_copy_key, __this__complexColumn_copy_value); + } + this.complexColumn = __this__complexColumn; + } + } + + public ParquetThriftCompat deepCopy() { + return new ParquetThriftCompat(this); + } + + @Override + public void clear() { + setBoolColumnIsSet(false); + this.boolColumn = false; + setByteColumnIsSet(false); + this.byteColumn = 0; + setShortColumnIsSet(false); + this.shortColumn = 0; + setIntColumnIsSet(false); + this.intColumn = 0; + setLongColumnIsSet(false); + this.longColumn = 0; + setDoubleColumnIsSet(false); + this.doubleColumn = 0.0; + this.binaryColumn = null; + this.stringColumn = null; + this.enumColumn = null; + setMaybeBoolColumnIsSet(false); + this.maybeBoolColumn = false; + setMaybeByteColumnIsSet(false); + this.maybeByteColumn = 0; + setMaybeShortColumnIsSet(false); + this.maybeShortColumn = 0; + setMaybeIntColumnIsSet(false); + this.maybeIntColumn = 0; + setMaybeLongColumnIsSet(false); + this.maybeLongColumn = 0; + setMaybeDoubleColumnIsSet(false); + this.maybeDoubleColumn = 0.0; + this.maybeBinaryColumn = null; + this.maybeStringColumn = null; + this.maybeEnumColumn = null; + this.stringsColumn = null; + this.intSetColumn = null; + this.intToStringColumn = null; + this.complexColumn = null; + } + + public boolean isBoolColumn() { + return this.boolColumn; + } + + public ParquetThriftCompat setBoolColumn(boolean boolColumn) { + this.boolColumn = boolColumn; + setBoolColumnIsSet(true); + return this; + } + + public void unsetBoolColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BOOLCOLUMN_ISSET_ID); + } + + /** Returns true if field boolColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetBoolColumn() { + return EncodingUtils.testBit(__isset_bitfield, __BOOLCOLUMN_ISSET_ID); + } + + public void setBoolColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BOOLCOLUMN_ISSET_ID, value); + } + + public byte getByteColumn() { + return this.byteColumn; + } + + public ParquetThriftCompat setByteColumn(byte byteColumn) { + this.byteColumn = byteColumn; + setByteColumnIsSet(true); + return this; + } + + public void unsetByteColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BYTECOLUMN_ISSET_ID); + } + + /** Returns true if field byteColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetByteColumn() { + return EncodingUtils.testBit(__isset_bitfield, __BYTECOLUMN_ISSET_ID); + } + + public void setByteColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BYTECOLUMN_ISSET_ID, value); + } + + public short getShortColumn() { + return this.shortColumn; + } + + public ParquetThriftCompat setShortColumn(short shortColumn) { + this.shortColumn = shortColumn; + setShortColumnIsSet(true); + return this; + } + + public void unsetShortColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SHORTCOLUMN_ISSET_ID); + } + + /** Returns true if field shortColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetShortColumn() { + return EncodingUtils.testBit(__isset_bitfield, __SHORTCOLUMN_ISSET_ID); + } + + public void setShortColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SHORTCOLUMN_ISSET_ID, value); + } + + public int getIntColumn() { + return this.intColumn; + } + + public ParquetThriftCompat setIntColumn(int intColumn) { + this.intColumn = intColumn; + setIntColumnIsSet(true); + return this; + } + + public void unsetIntColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __INTCOLUMN_ISSET_ID); + } + + /** Returns true if field intColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetIntColumn() { + return EncodingUtils.testBit(__isset_bitfield, __INTCOLUMN_ISSET_ID); + } + + public void setIntColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __INTCOLUMN_ISSET_ID, value); + } + + public long getLongColumn() { + return this.longColumn; + } + + public ParquetThriftCompat setLongColumn(long longColumn) { + this.longColumn = longColumn; + setLongColumnIsSet(true); + return this; + } + + public void unsetLongColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LONGCOLUMN_ISSET_ID); + } + + /** Returns true if field longColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetLongColumn() { + return EncodingUtils.testBit(__isset_bitfield, __LONGCOLUMN_ISSET_ID); + } + + public void setLongColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LONGCOLUMN_ISSET_ID, value); + } + + public double getDoubleColumn() { + return this.doubleColumn; + } + + public ParquetThriftCompat setDoubleColumn(double doubleColumn) { + this.doubleColumn = doubleColumn; + setDoubleColumnIsSet(true); + return this; + } + + public void unsetDoubleColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DOUBLECOLUMN_ISSET_ID); + } + + /** Returns true if field doubleColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetDoubleColumn() { + return EncodingUtils.testBit(__isset_bitfield, __DOUBLECOLUMN_ISSET_ID); + } + + public void setDoubleColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DOUBLECOLUMN_ISSET_ID, value); + } + + public byte[] getBinaryColumn() { + setBinaryColumn(org.apache.thrift.TBaseHelper.rightSize(binaryColumn)); + return binaryColumn == null ? null : binaryColumn.array(); + } + + public ByteBuffer bufferForBinaryColumn() { + return org.apache.thrift.TBaseHelper.copyBinary(binaryColumn); + } + + public ParquetThriftCompat setBinaryColumn(byte[] binaryColumn) { + this.binaryColumn = binaryColumn == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(binaryColumn, binaryColumn.length)); + return this; + } + + public ParquetThriftCompat setBinaryColumn(ByteBuffer binaryColumn) { + this.binaryColumn = org.apache.thrift.TBaseHelper.copyBinary(binaryColumn); + return this; + } + + public void unsetBinaryColumn() { + this.binaryColumn = null; + } + + /** Returns true if field binaryColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetBinaryColumn() { + return this.binaryColumn != null; + } + + public void setBinaryColumnIsSet(boolean value) { + if (!value) { + this.binaryColumn = null; + } + } + + public String getStringColumn() { + return this.stringColumn; + } + + public ParquetThriftCompat setStringColumn(String stringColumn) { + this.stringColumn = stringColumn; + return this; + } + + public void unsetStringColumn() { + this.stringColumn = null; + } + + /** Returns true if field stringColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetStringColumn() { + return this.stringColumn != null; + } + + public void setStringColumnIsSet(boolean value) { + if (!value) { + this.stringColumn = null; + } + } + + /** + * + * @see Suit + */ + public Suit getEnumColumn() { + return this.enumColumn; + } + + /** + * + * @see Suit + */ + public ParquetThriftCompat setEnumColumn(Suit enumColumn) { + this.enumColumn = enumColumn; + return this; + } + + public void unsetEnumColumn() { + this.enumColumn = null; + } + + /** Returns true if field enumColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetEnumColumn() { + return this.enumColumn != null; + } + + public void setEnumColumnIsSet(boolean value) { + if (!value) { + this.enumColumn = null; + } + } + + public boolean isMaybeBoolColumn() { + return this.maybeBoolColumn; + } + + public ParquetThriftCompat setMaybeBoolColumn(boolean maybeBoolColumn) { + this.maybeBoolColumn = maybeBoolColumn; + setMaybeBoolColumnIsSet(true); + return this; + } + + public void unsetMaybeBoolColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBEBOOLCOLUMN_ISSET_ID); + } + + /** Returns true if field maybeBoolColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeBoolColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBEBOOLCOLUMN_ISSET_ID); + } + + public void setMaybeBoolColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBEBOOLCOLUMN_ISSET_ID, value); + } + + public byte getMaybeByteColumn() { + return this.maybeByteColumn; + } + + public ParquetThriftCompat setMaybeByteColumn(byte maybeByteColumn) { + this.maybeByteColumn = maybeByteColumn; + setMaybeByteColumnIsSet(true); + return this; + } + + public void unsetMaybeByteColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBEBYTECOLUMN_ISSET_ID); + } + + /** Returns true if field maybeByteColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeByteColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBEBYTECOLUMN_ISSET_ID); + } + + public void setMaybeByteColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBEBYTECOLUMN_ISSET_ID, value); + } + + public short getMaybeShortColumn() { + return this.maybeShortColumn; + } + + public ParquetThriftCompat setMaybeShortColumn(short maybeShortColumn) { + this.maybeShortColumn = maybeShortColumn; + setMaybeShortColumnIsSet(true); + return this; + } + + public void unsetMaybeShortColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBESHORTCOLUMN_ISSET_ID); + } + + /** Returns true if field maybeShortColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeShortColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBESHORTCOLUMN_ISSET_ID); + } + + public void setMaybeShortColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBESHORTCOLUMN_ISSET_ID, value); + } + + public int getMaybeIntColumn() { + return this.maybeIntColumn; + } + + public ParquetThriftCompat setMaybeIntColumn(int maybeIntColumn) { + this.maybeIntColumn = maybeIntColumn; + setMaybeIntColumnIsSet(true); + return this; + } + + public void unsetMaybeIntColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBEINTCOLUMN_ISSET_ID); + } + + /** Returns true if field maybeIntColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeIntColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBEINTCOLUMN_ISSET_ID); + } + + public void setMaybeIntColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBEINTCOLUMN_ISSET_ID, value); + } + + public long getMaybeLongColumn() { + return this.maybeLongColumn; + } + + public ParquetThriftCompat setMaybeLongColumn(long maybeLongColumn) { + this.maybeLongColumn = maybeLongColumn; + setMaybeLongColumnIsSet(true); + return this; + } + + public void unsetMaybeLongColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBELONGCOLUMN_ISSET_ID); + } + + /** Returns true if field maybeLongColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeLongColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBELONGCOLUMN_ISSET_ID); + } + + public void setMaybeLongColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBELONGCOLUMN_ISSET_ID, value); + } + + public double getMaybeDoubleColumn() { + return this.maybeDoubleColumn; + } + + public ParquetThriftCompat setMaybeDoubleColumn(double maybeDoubleColumn) { + this.maybeDoubleColumn = maybeDoubleColumn; + setMaybeDoubleColumnIsSet(true); + return this; + } + + public void unsetMaybeDoubleColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBEDOUBLECOLUMN_ISSET_ID); + } + + /** Returns true if field maybeDoubleColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeDoubleColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBEDOUBLECOLUMN_ISSET_ID); + } + + public void setMaybeDoubleColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBEDOUBLECOLUMN_ISSET_ID, value); + } + + public byte[] getMaybeBinaryColumn() { + setMaybeBinaryColumn(org.apache.thrift.TBaseHelper.rightSize(maybeBinaryColumn)); + return maybeBinaryColumn == null ? null : maybeBinaryColumn.array(); + } + + public ByteBuffer bufferForMaybeBinaryColumn() { + return org.apache.thrift.TBaseHelper.copyBinary(maybeBinaryColumn); + } + + public ParquetThriftCompat setMaybeBinaryColumn(byte[] maybeBinaryColumn) { + this.maybeBinaryColumn = maybeBinaryColumn == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(maybeBinaryColumn, maybeBinaryColumn.length)); + return this; + } + + public ParquetThriftCompat setMaybeBinaryColumn(ByteBuffer maybeBinaryColumn) { + this.maybeBinaryColumn = org.apache.thrift.TBaseHelper.copyBinary(maybeBinaryColumn); + return this; + } + + public void unsetMaybeBinaryColumn() { + this.maybeBinaryColumn = null; + } + + /** Returns true if field maybeBinaryColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeBinaryColumn() { + return this.maybeBinaryColumn != null; + } + + public void setMaybeBinaryColumnIsSet(boolean value) { + if (!value) { + this.maybeBinaryColumn = null; + } + } + + public String getMaybeStringColumn() { + return this.maybeStringColumn; + } + + public ParquetThriftCompat setMaybeStringColumn(String maybeStringColumn) { + this.maybeStringColumn = maybeStringColumn; + return this; + } + + public void unsetMaybeStringColumn() { + this.maybeStringColumn = null; + } + + /** Returns true if field maybeStringColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeStringColumn() { + return this.maybeStringColumn != null; + } + + public void setMaybeStringColumnIsSet(boolean value) { + if (!value) { + this.maybeStringColumn = null; + } + } + + /** + * + * @see Suit + */ + public Suit getMaybeEnumColumn() { + return this.maybeEnumColumn; + } + + /** + * + * @see Suit + */ + public ParquetThriftCompat setMaybeEnumColumn(Suit maybeEnumColumn) { + this.maybeEnumColumn = maybeEnumColumn; + return this; + } + + public void unsetMaybeEnumColumn() { + this.maybeEnumColumn = null; + } + + /** Returns true if field maybeEnumColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeEnumColumn() { + return this.maybeEnumColumn != null; + } + + public void setMaybeEnumColumnIsSet(boolean value) { + if (!value) { + this.maybeEnumColumn = null; + } + } + + public int getStringsColumnSize() { + return (this.stringsColumn == null) ? 0 : this.stringsColumn.size(); + } + + public java.util.Iterator getStringsColumnIterator() { + return (this.stringsColumn == null) ? null : this.stringsColumn.iterator(); + } + + public void addToStringsColumn(String elem) { + if (this.stringsColumn == null) { + this.stringsColumn = new ArrayList(); + } + this.stringsColumn.add(elem); + } + + public List getStringsColumn() { + return this.stringsColumn; + } + + public ParquetThriftCompat setStringsColumn(List stringsColumn) { + this.stringsColumn = stringsColumn; + return this; + } + + public void unsetStringsColumn() { + this.stringsColumn = null; + } + + /** Returns true if field stringsColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetStringsColumn() { + return this.stringsColumn != null; + } + + public void setStringsColumnIsSet(boolean value) { + if (!value) { + this.stringsColumn = null; + } + } + + public int getIntSetColumnSize() { + return (this.intSetColumn == null) ? 0 : this.intSetColumn.size(); + } + + public java.util.Iterator getIntSetColumnIterator() { + return (this.intSetColumn == null) ? null : this.intSetColumn.iterator(); + } + + public void addToIntSetColumn(int elem) { + if (this.intSetColumn == null) { + this.intSetColumn = new HashSet(); + } + this.intSetColumn.add(elem); + } + + public Set getIntSetColumn() { + return this.intSetColumn; + } + + public ParquetThriftCompat setIntSetColumn(Set intSetColumn) { + this.intSetColumn = intSetColumn; + return this; + } + + public void unsetIntSetColumn() { + this.intSetColumn = null; + } + + /** Returns true if field intSetColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetIntSetColumn() { + return this.intSetColumn != null; + } + + public void setIntSetColumnIsSet(boolean value) { + if (!value) { + this.intSetColumn = null; + } + } + + public int getIntToStringColumnSize() { + return (this.intToStringColumn == null) ? 0 : this.intToStringColumn.size(); + } + + public void putToIntToStringColumn(int key, String val) { + if (this.intToStringColumn == null) { + this.intToStringColumn = new HashMap(); + } + this.intToStringColumn.put(key, val); + } + + public Map getIntToStringColumn() { + return this.intToStringColumn; + } + + public ParquetThriftCompat setIntToStringColumn(Map intToStringColumn) { + this.intToStringColumn = intToStringColumn; + return this; + } + + public void unsetIntToStringColumn() { + this.intToStringColumn = null; + } + + /** Returns true if field intToStringColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetIntToStringColumn() { + return this.intToStringColumn != null; + } + + public void setIntToStringColumnIsSet(boolean value) { + if (!value) { + this.intToStringColumn = null; + } + } + + public int getComplexColumnSize() { + return (this.complexColumn == null) ? 0 : this.complexColumn.size(); + } + + public void putToComplexColumn(int key, List val) { + if (this.complexColumn == null) { + this.complexColumn = new HashMap>(); + } + this.complexColumn.put(key, val); + } + + public Map> getComplexColumn() { + return this.complexColumn; + } + + public ParquetThriftCompat setComplexColumn(Map> complexColumn) { + this.complexColumn = complexColumn; + return this; + } + + public void unsetComplexColumn() { + this.complexColumn = null; + } + + /** Returns true if field complexColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetComplexColumn() { + return this.complexColumn != null; + } + + public void setComplexColumnIsSet(boolean value) { + if (!value) { + this.complexColumn = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case BOOL_COLUMN: + if (value == null) { + unsetBoolColumn(); + } else { + setBoolColumn((Boolean)value); + } + break; + + case BYTE_COLUMN: + if (value == null) { + unsetByteColumn(); + } else { + setByteColumn((Byte)value); + } + break; + + case SHORT_COLUMN: + if (value == null) { + unsetShortColumn(); + } else { + setShortColumn((Short)value); + } + break; + + case INT_COLUMN: + if (value == null) { + unsetIntColumn(); + } else { + setIntColumn((Integer)value); + } + break; + + case LONG_COLUMN: + if (value == null) { + unsetLongColumn(); + } else { + setLongColumn((Long)value); + } + break; + + case DOUBLE_COLUMN: + if (value == null) { + unsetDoubleColumn(); + } else { + setDoubleColumn((Double)value); + } + break; + + case BINARY_COLUMN: + if (value == null) { + unsetBinaryColumn(); + } else { + setBinaryColumn((ByteBuffer)value); + } + break; + + case STRING_COLUMN: + if (value == null) { + unsetStringColumn(); + } else { + setStringColumn((String)value); + } + break; + + case ENUM_COLUMN: + if (value == null) { + unsetEnumColumn(); + } else { + setEnumColumn((Suit)value); + } + break; + + case MAYBE_BOOL_COLUMN: + if (value == null) { + unsetMaybeBoolColumn(); + } else { + setMaybeBoolColumn((Boolean)value); + } + break; + + case MAYBE_BYTE_COLUMN: + if (value == null) { + unsetMaybeByteColumn(); + } else { + setMaybeByteColumn((Byte)value); + } + break; + + case MAYBE_SHORT_COLUMN: + if (value == null) { + unsetMaybeShortColumn(); + } else { + setMaybeShortColumn((Short)value); + } + break; + + case MAYBE_INT_COLUMN: + if (value == null) { + unsetMaybeIntColumn(); + } else { + setMaybeIntColumn((Integer)value); + } + break; + + case MAYBE_LONG_COLUMN: + if (value == null) { + unsetMaybeLongColumn(); + } else { + setMaybeLongColumn((Long)value); + } + break; + + case MAYBE_DOUBLE_COLUMN: + if (value == null) { + unsetMaybeDoubleColumn(); + } else { + setMaybeDoubleColumn((Double)value); + } + break; + + case MAYBE_BINARY_COLUMN: + if (value == null) { + unsetMaybeBinaryColumn(); + } else { + setMaybeBinaryColumn((ByteBuffer)value); + } + break; + + case MAYBE_STRING_COLUMN: + if (value == null) { + unsetMaybeStringColumn(); + } else { + setMaybeStringColumn((String)value); + } + break; + + case MAYBE_ENUM_COLUMN: + if (value == null) { + unsetMaybeEnumColumn(); + } else { + setMaybeEnumColumn((Suit)value); + } + break; + + case STRINGS_COLUMN: + if (value == null) { + unsetStringsColumn(); + } else { + setStringsColumn((List)value); + } + break; + + case INT_SET_COLUMN: + if (value == null) { + unsetIntSetColumn(); + } else { + setIntSetColumn((Set)value); + } + break; + + case INT_TO_STRING_COLUMN: + if (value == null) { + unsetIntToStringColumn(); + } else { + setIntToStringColumn((Map)value); + } + break; + + case COMPLEX_COLUMN: + if (value == null) { + unsetComplexColumn(); + } else { + setComplexColumn((Map>)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case BOOL_COLUMN: + return Boolean.valueOf(isBoolColumn()); + + case BYTE_COLUMN: + return Byte.valueOf(getByteColumn()); + + case SHORT_COLUMN: + return Short.valueOf(getShortColumn()); + + case INT_COLUMN: + return Integer.valueOf(getIntColumn()); + + case LONG_COLUMN: + return Long.valueOf(getLongColumn()); + + case DOUBLE_COLUMN: + return Double.valueOf(getDoubleColumn()); + + case BINARY_COLUMN: + return getBinaryColumn(); + + case STRING_COLUMN: + return getStringColumn(); + + case ENUM_COLUMN: + return getEnumColumn(); + + case MAYBE_BOOL_COLUMN: + return Boolean.valueOf(isMaybeBoolColumn()); + + case MAYBE_BYTE_COLUMN: + return Byte.valueOf(getMaybeByteColumn()); + + case MAYBE_SHORT_COLUMN: + return Short.valueOf(getMaybeShortColumn()); + + case MAYBE_INT_COLUMN: + return Integer.valueOf(getMaybeIntColumn()); + + case MAYBE_LONG_COLUMN: + return Long.valueOf(getMaybeLongColumn()); + + case MAYBE_DOUBLE_COLUMN: + return Double.valueOf(getMaybeDoubleColumn()); + + case MAYBE_BINARY_COLUMN: + return getMaybeBinaryColumn(); + + case MAYBE_STRING_COLUMN: + return getMaybeStringColumn(); + + case MAYBE_ENUM_COLUMN: + return getMaybeEnumColumn(); + + case STRINGS_COLUMN: + return getStringsColumn(); + + case INT_SET_COLUMN: + return getIntSetColumn(); + + case INT_TO_STRING_COLUMN: + return getIntToStringColumn(); + + case COMPLEX_COLUMN: + return getComplexColumn(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case BOOL_COLUMN: + return isSetBoolColumn(); + case BYTE_COLUMN: + return isSetByteColumn(); + case SHORT_COLUMN: + return isSetShortColumn(); + case INT_COLUMN: + return isSetIntColumn(); + case LONG_COLUMN: + return isSetLongColumn(); + case DOUBLE_COLUMN: + return isSetDoubleColumn(); + case BINARY_COLUMN: + return isSetBinaryColumn(); + case STRING_COLUMN: + return isSetStringColumn(); + case ENUM_COLUMN: + return isSetEnumColumn(); + case MAYBE_BOOL_COLUMN: + return isSetMaybeBoolColumn(); + case MAYBE_BYTE_COLUMN: + return isSetMaybeByteColumn(); + case MAYBE_SHORT_COLUMN: + return isSetMaybeShortColumn(); + case MAYBE_INT_COLUMN: + return isSetMaybeIntColumn(); + case MAYBE_LONG_COLUMN: + return isSetMaybeLongColumn(); + case MAYBE_DOUBLE_COLUMN: + return isSetMaybeDoubleColumn(); + case MAYBE_BINARY_COLUMN: + return isSetMaybeBinaryColumn(); + case MAYBE_STRING_COLUMN: + return isSetMaybeStringColumn(); + case MAYBE_ENUM_COLUMN: + return isSetMaybeEnumColumn(); + case STRINGS_COLUMN: + return isSetStringsColumn(); + case INT_SET_COLUMN: + return isSetIntSetColumn(); + case INT_TO_STRING_COLUMN: + return isSetIntToStringColumn(); + case COMPLEX_COLUMN: + return isSetComplexColumn(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ParquetThriftCompat) + return this.equals((ParquetThriftCompat)that); + return false; + } + + public boolean equals(ParquetThriftCompat that) { + if (that == null) + return false; + + boolean this_present_boolColumn = true; + boolean that_present_boolColumn = true; + if (this_present_boolColumn || that_present_boolColumn) { + if (!(this_present_boolColumn && that_present_boolColumn)) + return false; + if (this.boolColumn != that.boolColumn) + return false; + } + + boolean this_present_byteColumn = true; + boolean that_present_byteColumn = true; + if (this_present_byteColumn || that_present_byteColumn) { + if (!(this_present_byteColumn && that_present_byteColumn)) + return false; + if (this.byteColumn != that.byteColumn) + return false; + } + + boolean this_present_shortColumn = true; + boolean that_present_shortColumn = true; + if (this_present_shortColumn || that_present_shortColumn) { + if (!(this_present_shortColumn && that_present_shortColumn)) + return false; + if (this.shortColumn != that.shortColumn) + return false; + } + + boolean this_present_intColumn = true; + boolean that_present_intColumn = true; + if (this_present_intColumn || that_present_intColumn) { + if (!(this_present_intColumn && that_present_intColumn)) + return false; + if (this.intColumn != that.intColumn) + return false; + } + + boolean this_present_longColumn = true; + boolean that_present_longColumn = true; + if (this_present_longColumn || that_present_longColumn) { + if (!(this_present_longColumn && that_present_longColumn)) + return false; + if (this.longColumn != that.longColumn) + return false; + } + + boolean this_present_doubleColumn = true; + boolean that_present_doubleColumn = true; + if (this_present_doubleColumn || that_present_doubleColumn) { + if (!(this_present_doubleColumn && that_present_doubleColumn)) + return false; + if (this.doubleColumn != that.doubleColumn) + return false; + } + + boolean this_present_binaryColumn = true && this.isSetBinaryColumn(); + boolean that_present_binaryColumn = true && that.isSetBinaryColumn(); + if (this_present_binaryColumn || that_present_binaryColumn) { + if (!(this_present_binaryColumn && that_present_binaryColumn)) + return false; + if (!this.binaryColumn.equals(that.binaryColumn)) + return false; + } + + boolean this_present_stringColumn = true && this.isSetStringColumn(); + boolean that_present_stringColumn = true && that.isSetStringColumn(); + if (this_present_stringColumn || that_present_stringColumn) { + if (!(this_present_stringColumn && that_present_stringColumn)) + return false; + if (!this.stringColumn.equals(that.stringColumn)) + return false; + } + + boolean this_present_enumColumn = true && this.isSetEnumColumn(); + boolean that_present_enumColumn = true && that.isSetEnumColumn(); + if (this_present_enumColumn || that_present_enumColumn) { + if (!(this_present_enumColumn && that_present_enumColumn)) + return false; + if (!this.enumColumn.equals(that.enumColumn)) + return false; + } + + boolean this_present_maybeBoolColumn = true && this.isSetMaybeBoolColumn(); + boolean that_present_maybeBoolColumn = true && that.isSetMaybeBoolColumn(); + if (this_present_maybeBoolColumn || that_present_maybeBoolColumn) { + if (!(this_present_maybeBoolColumn && that_present_maybeBoolColumn)) + return false; + if (this.maybeBoolColumn != that.maybeBoolColumn) + return false; + } + + boolean this_present_maybeByteColumn = true && this.isSetMaybeByteColumn(); + boolean that_present_maybeByteColumn = true && that.isSetMaybeByteColumn(); + if (this_present_maybeByteColumn || that_present_maybeByteColumn) { + if (!(this_present_maybeByteColumn && that_present_maybeByteColumn)) + return false; + if (this.maybeByteColumn != that.maybeByteColumn) + return false; + } + + boolean this_present_maybeShortColumn = true && this.isSetMaybeShortColumn(); + boolean that_present_maybeShortColumn = true && that.isSetMaybeShortColumn(); + if (this_present_maybeShortColumn || that_present_maybeShortColumn) { + if (!(this_present_maybeShortColumn && that_present_maybeShortColumn)) + return false; + if (this.maybeShortColumn != that.maybeShortColumn) + return false; + } + + boolean this_present_maybeIntColumn = true && this.isSetMaybeIntColumn(); + boolean that_present_maybeIntColumn = true && that.isSetMaybeIntColumn(); + if (this_present_maybeIntColumn || that_present_maybeIntColumn) { + if (!(this_present_maybeIntColumn && that_present_maybeIntColumn)) + return false; + if (this.maybeIntColumn != that.maybeIntColumn) + return false; + } + + boolean this_present_maybeLongColumn = true && this.isSetMaybeLongColumn(); + boolean that_present_maybeLongColumn = true && that.isSetMaybeLongColumn(); + if (this_present_maybeLongColumn || that_present_maybeLongColumn) { + if (!(this_present_maybeLongColumn && that_present_maybeLongColumn)) + return false; + if (this.maybeLongColumn != that.maybeLongColumn) + return false; + } + + boolean this_present_maybeDoubleColumn = true && this.isSetMaybeDoubleColumn(); + boolean that_present_maybeDoubleColumn = true && that.isSetMaybeDoubleColumn(); + if (this_present_maybeDoubleColumn || that_present_maybeDoubleColumn) { + if (!(this_present_maybeDoubleColumn && that_present_maybeDoubleColumn)) + return false; + if (this.maybeDoubleColumn != that.maybeDoubleColumn) + return false; + } + + boolean this_present_maybeBinaryColumn = true && this.isSetMaybeBinaryColumn(); + boolean that_present_maybeBinaryColumn = true && that.isSetMaybeBinaryColumn(); + if (this_present_maybeBinaryColumn || that_present_maybeBinaryColumn) { + if (!(this_present_maybeBinaryColumn && that_present_maybeBinaryColumn)) + return false; + if (!this.maybeBinaryColumn.equals(that.maybeBinaryColumn)) + return false; + } + + boolean this_present_maybeStringColumn = true && this.isSetMaybeStringColumn(); + boolean that_present_maybeStringColumn = true && that.isSetMaybeStringColumn(); + if (this_present_maybeStringColumn || that_present_maybeStringColumn) { + if (!(this_present_maybeStringColumn && that_present_maybeStringColumn)) + return false; + if (!this.maybeStringColumn.equals(that.maybeStringColumn)) + return false; + } + + boolean this_present_maybeEnumColumn = true && this.isSetMaybeEnumColumn(); + boolean that_present_maybeEnumColumn = true && that.isSetMaybeEnumColumn(); + if (this_present_maybeEnumColumn || that_present_maybeEnumColumn) { + if (!(this_present_maybeEnumColumn && that_present_maybeEnumColumn)) + return false; + if (!this.maybeEnumColumn.equals(that.maybeEnumColumn)) + return false; + } + + boolean this_present_stringsColumn = true && this.isSetStringsColumn(); + boolean that_present_stringsColumn = true && that.isSetStringsColumn(); + if (this_present_stringsColumn || that_present_stringsColumn) { + if (!(this_present_stringsColumn && that_present_stringsColumn)) + return false; + if (!this.stringsColumn.equals(that.stringsColumn)) + return false; + } + + boolean this_present_intSetColumn = true && this.isSetIntSetColumn(); + boolean that_present_intSetColumn = true && that.isSetIntSetColumn(); + if (this_present_intSetColumn || that_present_intSetColumn) { + if (!(this_present_intSetColumn && that_present_intSetColumn)) + return false; + if (!this.intSetColumn.equals(that.intSetColumn)) + return false; + } + + boolean this_present_intToStringColumn = true && this.isSetIntToStringColumn(); + boolean that_present_intToStringColumn = true && that.isSetIntToStringColumn(); + if (this_present_intToStringColumn || that_present_intToStringColumn) { + if (!(this_present_intToStringColumn && that_present_intToStringColumn)) + return false; + if (!this.intToStringColumn.equals(that.intToStringColumn)) + return false; + } + + boolean this_present_complexColumn = true && this.isSetComplexColumn(); + boolean that_present_complexColumn = true && that.isSetComplexColumn(); + if (this_present_complexColumn || that_present_complexColumn) { + if (!(this_present_complexColumn && that_present_complexColumn)) + return false; + if (!this.complexColumn.equals(that.complexColumn)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_boolColumn = true; + list.add(present_boolColumn); + if (present_boolColumn) + list.add(boolColumn); + + boolean present_byteColumn = true; + list.add(present_byteColumn); + if (present_byteColumn) + list.add(byteColumn); + + boolean present_shortColumn = true; + list.add(present_shortColumn); + if (present_shortColumn) + list.add(shortColumn); + + boolean present_intColumn = true; + list.add(present_intColumn); + if (present_intColumn) + list.add(intColumn); + + boolean present_longColumn = true; + list.add(present_longColumn); + if (present_longColumn) + list.add(longColumn); + + boolean present_doubleColumn = true; + list.add(present_doubleColumn); + if (present_doubleColumn) + list.add(doubleColumn); + + boolean present_binaryColumn = true && (isSetBinaryColumn()); + list.add(present_binaryColumn); + if (present_binaryColumn) + list.add(binaryColumn); + + boolean present_stringColumn = true && (isSetStringColumn()); + list.add(present_stringColumn); + if (present_stringColumn) + list.add(stringColumn); + + boolean present_enumColumn = true && (isSetEnumColumn()); + list.add(present_enumColumn); + if (present_enumColumn) + list.add(enumColumn.getValue()); + + boolean present_maybeBoolColumn = true && (isSetMaybeBoolColumn()); + list.add(present_maybeBoolColumn); + if (present_maybeBoolColumn) + list.add(maybeBoolColumn); + + boolean present_maybeByteColumn = true && (isSetMaybeByteColumn()); + list.add(present_maybeByteColumn); + if (present_maybeByteColumn) + list.add(maybeByteColumn); + + boolean present_maybeShortColumn = true && (isSetMaybeShortColumn()); + list.add(present_maybeShortColumn); + if (present_maybeShortColumn) + list.add(maybeShortColumn); + + boolean present_maybeIntColumn = true && (isSetMaybeIntColumn()); + list.add(present_maybeIntColumn); + if (present_maybeIntColumn) + list.add(maybeIntColumn); + + boolean present_maybeLongColumn = true && (isSetMaybeLongColumn()); + list.add(present_maybeLongColumn); + if (present_maybeLongColumn) + list.add(maybeLongColumn); + + boolean present_maybeDoubleColumn = true && (isSetMaybeDoubleColumn()); + list.add(present_maybeDoubleColumn); + if (present_maybeDoubleColumn) + list.add(maybeDoubleColumn); + + boolean present_maybeBinaryColumn = true && (isSetMaybeBinaryColumn()); + list.add(present_maybeBinaryColumn); + if (present_maybeBinaryColumn) + list.add(maybeBinaryColumn); + + boolean present_maybeStringColumn = true && (isSetMaybeStringColumn()); + list.add(present_maybeStringColumn); + if (present_maybeStringColumn) + list.add(maybeStringColumn); + + boolean present_maybeEnumColumn = true && (isSetMaybeEnumColumn()); + list.add(present_maybeEnumColumn); + if (present_maybeEnumColumn) + list.add(maybeEnumColumn.getValue()); + + boolean present_stringsColumn = true && (isSetStringsColumn()); + list.add(present_stringsColumn); + if (present_stringsColumn) + list.add(stringsColumn); + + boolean present_intSetColumn = true && (isSetIntSetColumn()); + list.add(present_intSetColumn); + if (present_intSetColumn) + list.add(intSetColumn); + + boolean present_intToStringColumn = true && (isSetIntToStringColumn()); + list.add(present_intToStringColumn); + if (present_intToStringColumn) + list.add(intToStringColumn); + + boolean present_complexColumn = true && (isSetComplexColumn()); + list.add(present_complexColumn); + if (present_complexColumn) + list.add(complexColumn); + + return list.hashCode(); + } + + @Override + public int compareTo(ParquetThriftCompat other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetBoolColumn()).compareTo(other.isSetBoolColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetBoolColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.boolColumn, other.boolColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetByteColumn()).compareTo(other.isSetByteColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetByteColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.byteColumn, other.byteColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetShortColumn()).compareTo(other.isSetShortColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetShortColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.shortColumn, other.shortColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIntColumn()).compareTo(other.isSetIntColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIntColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.intColumn, other.intColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetLongColumn()).compareTo(other.isSetLongColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLongColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.longColumn, other.longColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDoubleColumn()).compareTo(other.isSetDoubleColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDoubleColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.doubleColumn, other.doubleColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetBinaryColumn()).compareTo(other.isSetBinaryColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetBinaryColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.binaryColumn, other.binaryColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStringColumn()).compareTo(other.isSetStringColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStringColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stringColumn, other.stringColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetEnumColumn()).compareTo(other.isSetEnumColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetEnumColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enumColumn, other.enumColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeBoolColumn()).compareTo(other.isSetMaybeBoolColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeBoolColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeBoolColumn, other.maybeBoolColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeByteColumn()).compareTo(other.isSetMaybeByteColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeByteColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeByteColumn, other.maybeByteColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeShortColumn()).compareTo(other.isSetMaybeShortColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeShortColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeShortColumn, other.maybeShortColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeIntColumn()).compareTo(other.isSetMaybeIntColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeIntColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeIntColumn, other.maybeIntColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeLongColumn()).compareTo(other.isSetMaybeLongColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeLongColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeLongColumn, other.maybeLongColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeDoubleColumn()).compareTo(other.isSetMaybeDoubleColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeDoubleColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeDoubleColumn, other.maybeDoubleColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeBinaryColumn()).compareTo(other.isSetMaybeBinaryColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeBinaryColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeBinaryColumn, other.maybeBinaryColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeStringColumn()).compareTo(other.isSetMaybeStringColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeStringColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeStringColumn, other.maybeStringColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeEnumColumn()).compareTo(other.isSetMaybeEnumColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeEnumColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeEnumColumn, other.maybeEnumColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStringsColumn()).compareTo(other.isSetStringsColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStringsColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stringsColumn, other.stringsColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIntSetColumn()).compareTo(other.isSetIntSetColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIntSetColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.intSetColumn, other.intSetColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIntToStringColumn()).compareTo(other.isSetIntToStringColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIntToStringColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.intToStringColumn, other.intToStringColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetComplexColumn()).compareTo(other.isSetComplexColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetComplexColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.complexColumn, other.complexColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ParquetThriftCompat("); + boolean first = true; + + sb.append("boolColumn:"); + sb.append(this.boolColumn); + first = false; + if (!first) sb.append(", "); + sb.append("byteColumn:"); + sb.append(this.byteColumn); + first = false; + if (!first) sb.append(", "); + sb.append("shortColumn:"); + sb.append(this.shortColumn); + first = false; + if (!first) sb.append(", "); + sb.append("intColumn:"); + sb.append(this.intColumn); + first = false; + if (!first) sb.append(", "); + sb.append("longColumn:"); + sb.append(this.longColumn); + first = false; + if (!first) sb.append(", "); + sb.append("doubleColumn:"); + sb.append(this.doubleColumn); + first = false; + if (!first) sb.append(", "); + sb.append("binaryColumn:"); + if (this.binaryColumn == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.binaryColumn, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("stringColumn:"); + if (this.stringColumn == null) { + sb.append("null"); + } else { + sb.append(this.stringColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("enumColumn:"); + if (this.enumColumn == null) { + sb.append("null"); + } else { + sb.append(this.enumColumn); + } + first = false; + if (isSetMaybeBoolColumn()) { + if (!first) sb.append(", "); + sb.append("maybeBoolColumn:"); + sb.append(this.maybeBoolColumn); + first = false; + } + if (isSetMaybeByteColumn()) { + if (!first) sb.append(", "); + sb.append("maybeByteColumn:"); + sb.append(this.maybeByteColumn); + first = false; + } + if (isSetMaybeShortColumn()) { + if (!first) sb.append(", "); + sb.append("maybeShortColumn:"); + sb.append(this.maybeShortColumn); + first = false; + } + if (isSetMaybeIntColumn()) { + if (!first) sb.append(", "); + sb.append("maybeIntColumn:"); + sb.append(this.maybeIntColumn); + first = false; + } + if (isSetMaybeLongColumn()) { + if (!first) sb.append(", "); + sb.append("maybeLongColumn:"); + sb.append(this.maybeLongColumn); + first = false; + } + if (isSetMaybeDoubleColumn()) { + if (!first) sb.append(", "); + sb.append("maybeDoubleColumn:"); + sb.append(this.maybeDoubleColumn); + first = false; + } + if (isSetMaybeBinaryColumn()) { + if (!first) sb.append(", "); + sb.append("maybeBinaryColumn:"); + if (this.maybeBinaryColumn == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.maybeBinaryColumn, sb); + } + first = false; + } + if (isSetMaybeStringColumn()) { + if (!first) sb.append(", "); + sb.append("maybeStringColumn:"); + if (this.maybeStringColumn == null) { + sb.append("null"); + } else { + sb.append(this.maybeStringColumn); + } + first = false; + } + if (isSetMaybeEnumColumn()) { + if (!first) sb.append(", "); + sb.append("maybeEnumColumn:"); + if (this.maybeEnumColumn == null) { + sb.append("null"); + } else { + sb.append(this.maybeEnumColumn); + } + first = false; + } + if (!first) sb.append(", "); + sb.append("stringsColumn:"); + if (this.stringsColumn == null) { + sb.append("null"); + } else { + sb.append(this.stringsColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("intSetColumn:"); + if (this.intSetColumn == null) { + sb.append("null"); + } else { + sb.append(this.intSetColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("intToStringColumn:"); + if (this.intToStringColumn == null) { + sb.append("null"); + } else { + sb.append(this.intToStringColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("complexColumn:"); + if (this.complexColumn == null) { + sb.append("null"); + } else { + sb.append(this.complexColumn); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // alas, we cannot check 'boolColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'byteColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'shortColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'intColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'longColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'doubleColumn' because it's a primitive and you chose the non-beans generator. + if (binaryColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'binaryColumn' was not present! Struct: " + toString()); + } + if (stringColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'stringColumn' was not present! Struct: " + toString()); + } + if (enumColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'enumColumn' was not present! Struct: " + toString()); + } + if (stringsColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'stringsColumn' was not present! Struct: " + toString()); + } + if (intSetColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'intSetColumn' was not present! Struct: " + toString()); + } + if (intToStringColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'intToStringColumn' was not present! Struct: " + toString()); + } + if (complexColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'complexColumn' was not present! Struct: " + toString()); + } + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ParquetThriftCompatStandardSchemeFactory implements SchemeFactory { + public ParquetThriftCompatStandardScheme getScheme() { + return new ParquetThriftCompatStandardScheme(); + } + } + + private static class ParquetThriftCompatStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ParquetThriftCompat struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // BOOL_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.boolColumn = iprot.readBool(); + struct.setBoolColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // BYTE_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { + struct.byteColumn = iprot.readByte(); + struct.setByteColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SHORT_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I16) { + struct.shortColumn = iprot.readI16(); + struct.setShortColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // INT_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.intColumn = iprot.readI32(); + struct.setIntColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // LONG_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.longColumn = iprot.readI64(); + struct.setLongColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // DOUBLE_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.doubleColumn = iprot.readDouble(); + struct.setDoubleColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // BINARY_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.binaryColumn = iprot.readBinary(); + struct.setBinaryColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // STRING_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.stringColumn = iprot.readString(); + struct.setStringColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 9: // ENUM_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.enumColumn = org.apache.spark.sql.parquet.test.thrift.Suit.findByValue(iprot.readI32()); + struct.setEnumColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 10: // MAYBE_BOOL_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.maybeBoolColumn = iprot.readBool(); + struct.setMaybeBoolColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 11: // MAYBE_BYTE_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { + struct.maybeByteColumn = iprot.readByte(); + struct.setMaybeByteColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 12: // MAYBE_SHORT_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I16) { + struct.maybeShortColumn = iprot.readI16(); + struct.setMaybeShortColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 13: // MAYBE_INT_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.maybeIntColumn = iprot.readI32(); + struct.setMaybeIntColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 14: // MAYBE_LONG_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.maybeLongColumn = iprot.readI64(); + struct.setMaybeLongColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 15: // MAYBE_DOUBLE_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.maybeDoubleColumn = iprot.readDouble(); + struct.setMaybeDoubleColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 16: // MAYBE_BINARY_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.maybeBinaryColumn = iprot.readBinary(); + struct.setMaybeBinaryColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 17: // MAYBE_STRING_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.maybeStringColumn = iprot.readString(); + struct.setMaybeStringColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 18: // MAYBE_ENUM_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.maybeEnumColumn = org.apache.spark.sql.parquet.test.thrift.Suit.findByValue(iprot.readI32()); + struct.setMaybeEnumColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 19: // STRINGS_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list8 = iprot.readListBegin(); + struct.stringsColumn = new ArrayList(_list8.size); + String _elem9; + for (int _i10 = 0; _i10 < _list8.size; ++_i10) + { + _elem9 = iprot.readString(); + struct.stringsColumn.add(_elem9); + } + iprot.readListEnd(); + } + struct.setStringsColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 20: // INT_SET_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.SET) { + { + org.apache.thrift.protocol.TSet _set11 = iprot.readSetBegin(); + struct.intSetColumn = new HashSet(2*_set11.size); + int _elem12; + for (int _i13 = 0; _i13 < _set11.size; ++_i13) + { + _elem12 = iprot.readI32(); + struct.intSetColumn.add(_elem12); + } + iprot.readSetEnd(); + } + struct.setIntSetColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 21: // INT_TO_STRING_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map14 = iprot.readMapBegin(); + struct.intToStringColumn = new HashMap(2*_map14.size); + int _key15; + String _val16; + for (int _i17 = 0; _i17 < _map14.size; ++_i17) + { + _key15 = iprot.readI32(); + _val16 = iprot.readString(); + struct.intToStringColumn.put(_key15, _val16); + } + iprot.readMapEnd(); + } + struct.setIntToStringColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 22: // COMPLEX_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map18 = iprot.readMapBegin(); + struct.complexColumn = new HashMap>(2*_map18.size); + int _key19; + List _val20; + for (int _i21 = 0; _i21 < _map18.size; ++_i21) + { + _key19 = iprot.readI32(); + { + org.apache.thrift.protocol.TList _list22 = iprot.readListBegin(); + _val20 = new ArrayList(_list22.size); + Nested _elem23; + for (int _i24 = 0; _i24 < _list22.size; ++_i24) + { + _elem23 = new Nested(); + _elem23.read(iprot); + _val20.add(_elem23); + } + iprot.readListEnd(); + } + struct.complexColumn.put(_key19, _val20); + } + iprot.readMapEnd(); + } + struct.setComplexColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetBoolColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'boolColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetByteColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'byteColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetShortColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'shortColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetIntColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'intColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetLongColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'longColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetDoubleColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'doubleColumn' was not found in serialized data! Struct: " + toString()); + } + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ParquetThriftCompat struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(BOOL_COLUMN_FIELD_DESC); + oprot.writeBool(struct.boolColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(BYTE_COLUMN_FIELD_DESC); + oprot.writeByte(struct.byteColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(SHORT_COLUMN_FIELD_DESC); + oprot.writeI16(struct.shortColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(INT_COLUMN_FIELD_DESC); + oprot.writeI32(struct.intColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(LONG_COLUMN_FIELD_DESC); + oprot.writeI64(struct.longColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(DOUBLE_COLUMN_FIELD_DESC); + oprot.writeDouble(struct.doubleColumn); + oprot.writeFieldEnd(); + if (struct.binaryColumn != null) { + oprot.writeFieldBegin(BINARY_COLUMN_FIELD_DESC); + oprot.writeBinary(struct.binaryColumn); + oprot.writeFieldEnd(); + } + if (struct.stringColumn != null) { + oprot.writeFieldBegin(STRING_COLUMN_FIELD_DESC); + oprot.writeString(struct.stringColumn); + oprot.writeFieldEnd(); + } + if (struct.enumColumn != null) { + oprot.writeFieldBegin(ENUM_COLUMN_FIELD_DESC); + oprot.writeI32(struct.enumColumn.getValue()); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeBoolColumn()) { + oprot.writeFieldBegin(MAYBE_BOOL_COLUMN_FIELD_DESC); + oprot.writeBool(struct.maybeBoolColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeByteColumn()) { + oprot.writeFieldBegin(MAYBE_BYTE_COLUMN_FIELD_DESC); + oprot.writeByte(struct.maybeByteColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeShortColumn()) { + oprot.writeFieldBegin(MAYBE_SHORT_COLUMN_FIELD_DESC); + oprot.writeI16(struct.maybeShortColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeIntColumn()) { + oprot.writeFieldBegin(MAYBE_INT_COLUMN_FIELD_DESC); + oprot.writeI32(struct.maybeIntColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeLongColumn()) { + oprot.writeFieldBegin(MAYBE_LONG_COLUMN_FIELD_DESC); + oprot.writeI64(struct.maybeLongColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeDoubleColumn()) { + oprot.writeFieldBegin(MAYBE_DOUBLE_COLUMN_FIELD_DESC); + oprot.writeDouble(struct.maybeDoubleColumn); + oprot.writeFieldEnd(); + } + if (struct.maybeBinaryColumn != null) { + if (struct.isSetMaybeBinaryColumn()) { + oprot.writeFieldBegin(MAYBE_BINARY_COLUMN_FIELD_DESC); + oprot.writeBinary(struct.maybeBinaryColumn); + oprot.writeFieldEnd(); + } + } + if (struct.maybeStringColumn != null) { + if (struct.isSetMaybeStringColumn()) { + oprot.writeFieldBegin(MAYBE_STRING_COLUMN_FIELD_DESC); + oprot.writeString(struct.maybeStringColumn); + oprot.writeFieldEnd(); + } + } + if (struct.maybeEnumColumn != null) { + if (struct.isSetMaybeEnumColumn()) { + oprot.writeFieldBegin(MAYBE_ENUM_COLUMN_FIELD_DESC); + oprot.writeI32(struct.maybeEnumColumn.getValue()); + oprot.writeFieldEnd(); + } + } + if (struct.stringsColumn != null) { + oprot.writeFieldBegin(STRINGS_COLUMN_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.stringsColumn.size())); + for (String _iter25 : struct.stringsColumn) + { + oprot.writeString(_iter25); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.intSetColumn != null) { + oprot.writeFieldBegin(INT_SET_COLUMN_FIELD_DESC); + { + oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I32, struct.intSetColumn.size())); + for (int _iter26 : struct.intSetColumn) + { + oprot.writeI32(_iter26); + } + oprot.writeSetEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.intToStringColumn != null) { + oprot.writeFieldBegin(INT_TO_STRING_COLUMN_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRING, struct.intToStringColumn.size())); + for (Map.Entry _iter27 : struct.intToStringColumn.entrySet()) + { + oprot.writeI32(_iter27.getKey()); + oprot.writeString(_iter27.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.complexColumn != null) { + oprot.writeFieldBegin(COMPLEX_COLUMN_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.LIST, struct.complexColumn.size())); + for (Map.Entry> _iter28 : struct.complexColumn.entrySet()) + { + oprot.writeI32(_iter28.getKey()); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter28.getValue().size())); + for (Nested _iter29 : _iter28.getValue()) + { + _iter29.write(oprot); + } + oprot.writeListEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ParquetThriftCompatTupleSchemeFactory implements SchemeFactory { + public ParquetThriftCompatTupleScheme getScheme() { + return new ParquetThriftCompatTupleScheme(); + } + } + + private static class ParquetThriftCompatTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ParquetThriftCompat struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeBool(struct.boolColumn); + oprot.writeByte(struct.byteColumn); + oprot.writeI16(struct.shortColumn); + oprot.writeI32(struct.intColumn); + oprot.writeI64(struct.longColumn); + oprot.writeDouble(struct.doubleColumn); + oprot.writeBinary(struct.binaryColumn); + oprot.writeString(struct.stringColumn); + oprot.writeI32(struct.enumColumn.getValue()); + { + oprot.writeI32(struct.stringsColumn.size()); + for (String _iter30 : struct.stringsColumn) + { + oprot.writeString(_iter30); + } + } + { + oprot.writeI32(struct.intSetColumn.size()); + for (int _iter31 : struct.intSetColumn) + { + oprot.writeI32(_iter31); + } + } + { + oprot.writeI32(struct.intToStringColumn.size()); + for (Map.Entry _iter32 : struct.intToStringColumn.entrySet()) + { + oprot.writeI32(_iter32.getKey()); + oprot.writeString(_iter32.getValue()); + } + } + { + oprot.writeI32(struct.complexColumn.size()); + for (Map.Entry> _iter33 : struct.complexColumn.entrySet()) + { + oprot.writeI32(_iter33.getKey()); + { + oprot.writeI32(_iter33.getValue().size()); + for (Nested _iter34 : _iter33.getValue()) + { + _iter34.write(oprot); + } + } + } + } + BitSet optionals = new BitSet(); + if (struct.isSetMaybeBoolColumn()) { + optionals.set(0); + } + if (struct.isSetMaybeByteColumn()) { + optionals.set(1); + } + if (struct.isSetMaybeShortColumn()) { + optionals.set(2); + } + if (struct.isSetMaybeIntColumn()) { + optionals.set(3); + } + if (struct.isSetMaybeLongColumn()) { + optionals.set(4); + } + if (struct.isSetMaybeDoubleColumn()) { + optionals.set(5); + } + if (struct.isSetMaybeBinaryColumn()) { + optionals.set(6); + } + if (struct.isSetMaybeStringColumn()) { + optionals.set(7); + } + if (struct.isSetMaybeEnumColumn()) { + optionals.set(8); + } + oprot.writeBitSet(optionals, 9); + if (struct.isSetMaybeBoolColumn()) { + oprot.writeBool(struct.maybeBoolColumn); + } + if (struct.isSetMaybeByteColumn()) { + oprot.writeByte(struct.maybeByteColumn); + } + if (struct.isSetMaybeShortColumn()) { + oprot.writeI16(struct.maybeShortColumn); + } + if (struct.isSetMaybeIntColumn()) { + oprot.writeI32(struct.maybeIntColumn); + } + if (struct.isSetMaybeLongColumn()) { + oprot.writeI64(struct.maybeLongColumn); + } + if (struct.isSetMaybeDoubleColumn()) { + oprot.writeDouble(struct.maybeDoubleColumn); + } + if (struct.isSetMaybeBinaryColumn()) { + oprot.writeBinary(struct.maybeBinaryColumn); + } + if (struct.isSetMaybeStringColumn()) { + oprot.writeString(struct.maybeStringColumn); + } + if (struct.isSetMaybeEnumColumn()) { + oprot.writeI32(struct.maybeEnumColumn.getValue()); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ParquetThriftCompat struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.boolColumn = iprot.readBool(); + struct.setBoolColumnIsSet(true); + struct.byteColumn = iprot.readByte(); + struct.setByteColumnIsSet(true); + struct.shortColumn = iprot.readI16(); + struct.setShortColumnIsSet(true); + struct.intColumn = iprot.readI32(); + struct.setIntColumnIsSet(true); + struct.longColumn = iprot.readI64(); + struct.setLongColumnIsSet(true); + struct.doubleColumn = iprot.readDouble(); + struct.setDoubleColumnIsSet(true); + struct.binaryColumn = iprot.readBinary(); + struct.setBinaryColumnIsSet(true); + struct.stringColumn = iprot.readString(); + struct.setStringColumnIsSet(true); + struct.enumColumn = org.apache.spark.sql.parquet.test.thrift.Suit.findByValue(iprot.readI32()); + struct.setEnumColumnIsSet(true); + { + org.apache.thrift.protocol.TList _list35 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.stringsColumn = new ArrayList(_list35.size); + String _elem36; + for (int _i37 = 0; _i37 < _list35.size; ++_i37) + { + _elem36 = iprot.readString(); + struct.stringsColumn.add(_elem36); + } + } + struct.setStringsColumnIsSet(true); + { + org.apache.thrift.protocol.TSet _set38 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.intSetColumn = new HashSet(2*_set38.size); + int _elem39; + for (int _i40 = 0; _i40 < _set38.size; ++_i40) + { + _elem39 = iprot.readI32(); + struct.intSetColumn.add(_elem39); + } + } + struct.setIntSetColumnIsSet(true); + { + org.apache.thrift.protocol.TMap _map41 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.intToStringColumn = new HashMap(2*_map41.size); + int _key42; + String _val43; + for (int _i44 = 0; _i44 < _map41.size; ++_i44) + { + _key42 = iprot.readI32(); + _val43 = iprot.readString(); + struct.intToStringColumn.put(_key42, _val43); + } + } + struct.setIntToStringColumnIsSet(true); + { + org.apache.thrift.protocol.TMap _map45 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.LIST, iprot.readI32()); + struct.complexColumn = new HashMap>(2*_map45.size); + int _key46; + List _val47; + for (int _i48 = 0; _i48 < _map45.size; ++_i48) + { + _key46 = iprot.readI32(); + { + org.apache.thrift.protocol.TList _list49 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + _val47 = new ArrayList(_list49.size); + Nested _elem50; + for (int _i51 = 0; _i51 < _list49.size; ++_i51) + { + _elem50 = new Nested(); + _elem50.read(iprot); + _val47.add(_elem50); + } + } + struct.complexColumn.put(_key46, _val47); + } + } + struct.setComplexColumnIsSet(true); + BitSet incoming = iprot.readBitSet(9); + if (incoming.get(0)) { + struct.maybeBoolColumn = iprot.readBool(); + struct.setMaybeBoolColumnIsSet(true); + } + if (incoming.get(1)) { + struct.maybeByteColumn = iprot.readByte(); + struct.setMaybeByteColumnIsSet(true); + } + if (incoming.get(2)) { + struct.maybeShortColumn = iprot.readI16(); + struct.setMaybeShortColumnIsSet(true); + } + if (incoming.get(3)) { + struct.maybeIntColumn = iprot.readI32(); + struct.setMaybeIntColumnIsSet(true); + } + if (incoming.get(4)) { + struct.maybeLongColumn = iprot.readI64(); + struct.setMaybeLongColumnIsSet(true); + } + if (incoming.get(5)) { + struct.maybeDoubleColumn = iprot.readDouble(); + struct.setMaybeDoubleColumnIsSet(true); + } + if (incoming.get(6)) { + struct.maybeBinaryColumn = iprot.readBinary(); + struct.setMaybeBinaryColumnIsSet(true); + } + if (incoming.get(7)) { + struct.maybeStringColumn = iprot.readString(); + struct.setMaybeStringColumnIsSet(true); + } + if (incoming.get(8)) { + struct.maybeEnumColumn = org.apache.spark.sql.parquet.test.thrift.Suit.findByValue(iprot.readI32()); + struct.setMaybeEnumColumnIsSet(true); + } + } + } + +} + diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Suit.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Suit.java new file mode 100644 index 0000000000000..5315c6aae9372 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Suit.java @@ -0,0 +1,51 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.spark.sql.parquet.test.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum Suit implements org.apache.thrift.TEnum { + SPADES(0), + HEARTS(1), + DIAMONDS(2), + CLUBS(3); + + private final int value; + + private Suit(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static Suit findByValue(int value) { + switch (value) { + case 0: + return SPADES; + case 1: + return HEARTS; + case 2: + return DIAMONDS; + case 3: + return CLUBS; + default: + return null; + } + } +} 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 new file mode 100644 index 0000000000000..bfa427349ff6a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala @@ -0,0 +1,125 @@ +/* + * 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.ByteBuffer +import java.util.{List => JList, Map => JMap} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path +import org.apache.parquet.avro.AvroParquetWriter + +import org.apache.spark.sql.parquet.test.avro.{Nested, ParquetAvroCompat} +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.{Row, SQLContext} + +class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest { + import ParquetCompatibilityTest._ + + override val sqlContext: SQLContext = TestSQLContext + + override protected def beforeAll(): Unit = { + super.beforeAll() + + val writer = + new AvroParquetWriter[ParquetAvroCompat]( + new Path(parquetStore.getCanonicalPath), + ParquetAvroCompat.getClassSchema) + + (0 until 10).foreach(i => writer.write(makeParquetAvroCompat(i))) + writer.close() + } + + test("Read Parquet file generated by parquet-avro") { + logInfo( + s"""Schema of the Parquet file written by parquet-avro: + |${readParquetSchema(parquetStore.getCanonicalPath)} + """.stripMargin) + + checkAnswer(sqlContext.read.parquet(parquetStore.getCanonicalPath), (0 until 10).map { i => + def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) + + Row( + i % 2 == 0, + i, + i.toLong * 10, + i.toFloat + 0.1f, + i.toDouble + 0.2d, + s"val_$i".getBytes, + s"val_$i", + + nullable(i % 2 == 0: java.lang.Boolean), + nullable(i: Integer), + nullable(i.toLong: java.lang.Long), + nullable(i.toFloat + 0.1f: java.lang.Float), + nullable(i.toDouble + 0.2d: java.lang.Double), + nullable(s"val_$i".getBytes), + nullable(s"val_$i"), + + Seq.tabulate(3)(n => s"arr_${i + n}"), + Seq.tabulate(3)(n => n.toString -> (i + n: Integer)).toMap, + Seq.tabulate(3) { n => + (i + n).toString -> Seq.tabulate(3) { m => + Row(Seq.tabulate(3)(j => i + j + m), s"val_${i + m}") + } + }.toMap) + }) + } + + def makeParquetAvroCompat(i: Int): ParquetAvroCompat = { + def nullable[T <: AnyRef] = makeNullable[T](i) _ + + def makeComplexColumn(i: Int): JMap[String, JList[Nested]] = { + mapAsJavaMap(Seq.tabulate(3) { n => + (i + n).toString -> seqAsJavaList(Seq.tabulate(3) { m => + Nested + .newBuilder() + .setNestedIntsColumn(seqAsJavaList(Seq.tabulate(3)(j => i + j + m))) + .setNestedStringColumn(s"val_${i + m}") + .build() + }) + }.toMap) + } + + ParquetAvroCompat + .newBuilder() + .setBoolColumn(i % 2 == 0) + .setIntColumn(i) + .setLongColumn(i.toLong * 10) + .setFloatColumn(i.toFloat + 0.1f) + .setDoubleColumn(i.toDouble + 0.2d) + .setBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes)) + .setStringColumn(s"val_$i") + + .setMaybeBoolColumn(nullable(i % 2 == 0: java.lang.Boolean)) + .setMaybeIntColumn(nullable(i: Integer)) + .setMaybeLongColumn(nullable(i.toLong: java.lang.Long)) + .setMaybeFloatColumn(nullable(i.toFloat + 0.1f: java.lang.Float)) + .setMaybeDoubleColumn(nullable(i.toDouble + 0.2d: java.lang.Double)) + .setMaybeBinaryColumn(nullable(ByteBuffer.wrap(s"val_$i".getBytes))) + .setMaybeStringColumn(nullable(s"val_$i")) + + .setStringsColumn(Seq.tabulate(3)(n => s"arr_${i + n}")) + .setStringToIntColumn( + mapAsJavaMap(Seq.tabulate(3)(n => n.toString -> (i + n: Integer)).toMap)) + .setComplexColumn(makeComplexColumn(i)) + + .build() + } +} 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 new file mode 100644 index 0000000000000..b4cdfd9e98f6f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.parquet +import java.io.File + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.schema.MessageType +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.QueryTest +import org.apache.spark.util.Utils + +abstract class ParquetCompatibilityTest extends QueryTest with ParquetTest with BeforeAndAfterAll { + protected var parquetStore: File = _ + + override protected def beforeAll(): Unit = { + parquetStore = Utils.createTempDir(namePrefix = "parquet-compat_") + parquetStore.delete() + } + + override protected def afterAll(): Unit = { + Utils.deleteRecursively(parquetStore) + } + + def readParquetSchema(path: String): MessageType = { + val fsPath = new Path(path) + 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 + } +} + +object ParquetCompatibilityTest { + def makeNullable[T <: AnyRef](i: Int)(f: => T): T = { + if (i % 3 == 0) null.asInstanceOf[T] else f + } +} 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 new file mode 100644 index 0000000000000..d22066cabc567 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetThriftCompatibilitySuite.scala @@ -0,0 +1,140 @@ +/* + * 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.ByteBuffer +import java.util.{List => JList, Map => JMap} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.apache.parquet.thrift.ThriftParquetWriter + +import org.apache.spark.sql.parquet.test.thrift.{Nested, ParquetThriftCompat, Suit} +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.{Row, SQLContext} + +class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest { + import ParquetCompatibilityTest._ + + override val sqlContext: SQLContext = TestSQLContext + + override protected def beforeAll(): Unit = { + super.beforeAll() + + val writer = + new ThriftParquetWriter[ParquetThriftCompat]( + new Path(parquetStore.getCanonicalPath), + classOf[ParquetThriftCompat], + CompressionCodecName.SNAPPY) + + (0 until 10).foreach(i => writer.write(makeParquetThriftCompat(i))) + writer.close() + } + + test("Read Parquet file generated by parquet-thrift") { + logInfo( + s"""Schema of the Parquet file written by parquet-thrift: + |${readParquetSchema(parquetStore.getCanonicalPath)} + """.stripMargin) + + checkAnswer(sqlContext.read.parquet(parquetStore.getCanonicalPath), (0 until 10).map { i => + def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) + + Row( + i % 2 == 0, + i.toByte, + (i + 1).toShort, + i + 2, + i.toLong * 10, + i.toDouble + 0.2d, + // Thrift `BINARY` values are actually unencoded `STRING` values, and thus are always + // treated as `BINARY (UTF8)` in parquet-thrift, since parquet-thrift always assume + // Thrift `STRING`s are encoded using UTF-8. + s"val_$i", + s"val_$i", + // Thrift ENUM values are converted to Parquet binaries containing UTF-8 strings + Suit.values()(i % 4).name(), + + 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 * 10).toLong: java.lang.Long), + nullable(i.toDouble + 0.2d: java.lang.Double), + nullable(s"val_$i"), + nullable(s"val_$i"), + nullable(Suit.values()(i % 4).name()), + + Seq.tabulate(3)(n => s"arr_${i + n}"), + // Thrift `SET`s are converted to Parquet `LIST`s + Seq(i), + Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap, + Seq.tabulate(3) { n => + (i + n) -> Seq.tabulate(3) { m => + Row(Seq.tabulate(3)(j => i + j + m), s"val_${i + m}") + } + }.toMap) + }) + } + + def makeParquetThriftCompat(i: Int): ParquetThriftCompat = { + def makeComplexColumn(i: Int): JMap[Integer, JList[Nested]] = { + mapAsJavaMap(Seq.tabulate(3) { n => + (i + n: Integer) -> seqAsJavaList(Seq.tabulate(3) { m => + new Nested( + seqAsJavaList(Seq.tabulate(3)(j => i + j + m)), + s"val_${i + m}") + }) + }.toMap) + } + + val value = + new ParquetThriftCompat( + i % 2 == 0, + i.toByte, + (i + 1).toShort, + i + 2, + i.toLong * 10, + i.toDouble + 0.2d, + ByteBuffer.wrap(s"val_$i".getBytes), + s"val_$i", + Suit.values()(i % 4), + + seqAsJavaList(Seq.tabulate(3)(n => s"arr_${i + n}")), + setAsJavaSet(Set(i)), + mapAsJavaMap(Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap), + makeComplexColumn(i)) + + if (i % 3 == 0) { + value + } else { + value + .setMaybeBoolColumn(i % 2 == 0) + .setMaybeByteColumn(i.toByte) + .setMaybeShortColumn((i + 1).toShort) + .setMaybeIntColumn(i + 2) + .setMaybeLongColumn(i.toLong * 10) + .setMaybeDoubleColumn(i.toDouble + 0.2d) + .setMaybeBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes)) + .setMaybeStringColumn(s"val_$i") + .setMaybeEnumColumn(Suit.values()(i % 4)) + } + } +} diff --git a/sql/core/src/test/scripts/gen-code.sh b/sql/core/src/test/scripts/gen-code.sh new file mode 100755 index 0000000000000..5d8d8ad08555c --- /dev/null +++ b/sql/core/src/test/scripts/gen-code.sh @@ -0,0 +1,31 @@ +# +# 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. +# + +cd $(dirname $0)/.. +BASEDIR=`pwd` +cd - + +rm -rf $BASEDIR/gen-java +mkdir -p $BASEDIR/gen-java + +thrift\ + --gen java\ + -out $BASEDIR/gen-java\ + $BASEDIR/thrift/parquet-compat.thrift + +avro-tools idl $BASEDIR/avro/parquet-compat.avdl > $BASEDIR/avro/parquet-compat.avpr +avro-tools compile -string protocol $BASEDIR/avro/parquet-compat.avpr $BASEDIR/gen-java diff --git a/sql/core/src/test/thrift/parquet-compat.thrift b/sql/core/src/test/thrift/parquet-compat.thrift new file mode 100644 index 0000000000000..fa5ed8c62306a --- /dev/null +++ b/sql/core/src/test/thrift/parquet-compat.thrift @@ -0,0 +1,60 @@ +/* + * 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. + */ + +namespace java org.apache.spark.sql.parquet.test.thrift + +enum Suit { + SPADES, + HEARTS, + DIAMONDS, + CLUBS +} + +struct Nested { + 1: required list nestedIntsColumn; + 2: required string nestedStringColumn; +} + +/** + * This is a test struct for testing parquet-thrift compatibility. + */ +struct ParquetThriftCompat { + 1: required bool boolColumn; + 2: required byte byteColumn; + 3: required i16 shortColumn; + 4: required i32 intColumn; + 5: required i64 longColumn; + 6: required double doubleColumn; + 7: required binary binaryColumn; + 8: required string stringColumn; + 9: required Suit enumColumn + + 10: optional bool maybeBoolColumn; + 11: optional byte maybeByteColumn; + 12: optional i16 maybeShortColumn; + 13: optional i32 maybeIntColumn; + 14: optional i64 maybeLongColumn; + 15: optional double maybeDoubleColumn; + 16: optional binary maybeBinaryColumn; + 17: optional string maybeStringColumn; + 18: optional Suit maybeEnumColumn; + + 19: required list stringsColumn; + 20: required set intSetColumn; + 21: required map intToStringColumn; + 22: required map> complexColumn; +} 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)) + } + } +} 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..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,14 +21,16 @@ 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 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 // The data where the partitioning key exists only in the directory structure. @@ -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: SQLContext = TestHive + var partitionedTableDir: File = null var normalTableDir: File = null var partitionedTableDirWithKey: File = null