From aa688fe6e2da761f16b45ff860f543709d989ecc Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 26 Mar 2014 09:46:10 +0200 Subject: [PATCH 01/48] Adding conversion of nested Parquet schemas --- .../spark/sql/parquet/ParquetRelation.scala | 90 ++++++++++++----- .../spark/sql/parquet/ParquetTestData.scala | 96 ++++++++++++++++++- .../spark/sql/parquet/ParquetQuerySuite.scala | 37 +++++++ 3 files changed, 196 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 32813a66de3c3..90ce1d7076b4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.parquet -import java.io.IOException +import java.io.IOException, import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -26,9 +26,10 @@ import org.apache.hadoop.mapreduce.Job import parquet.hadoop.util.ContextUtil import parquet.hadoop.{ParquetOutputFormat, Footer, ParquetFileWriter, ParquetFileReader} + import parquet.hadoop.metadata.{CompressionCodecName, FileMetaData, ParquetMetadata} import parquet.io.api.{Binary, RecordConsumer} -import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser} +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser, GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import parquet.schema.Type.Repetition @@ -172,7 +173,7 @@ private[sql] object ParquetRelation { } private[parquet] object ParquetTypesConverter { - def toDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { + def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { // for now map binary to string type // TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema case ParquetPrimitiveTypeName.BINARY => StringType @@ -190,15 +191,61 @@ private[parquet] object ParquetTypesConverter { s"Unsupported parquet datatype $parquetType") } - def fromDataType(ctype: DataType): ParquetPrimitiveTypeName = ctype match { - case StringType => ParquetPrimitiveTypeName.BINARY - case BooleanType => ParquetPrimitiveTypeName.BOOLEAN - case DoubleType => ParquetPrimitiveTypeName.DOUBLE - case ArrayType(ByteType) => ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY - case FloatType => ParquetPrimitiveTypeName.FLOAT - case IntegerType => ParquetPrimitiveTypeName.INT32 - case LongType => ParquetPrimitiveTypeName.INT64 - case _ => sys.error(s"Unsupported datatype $ctype") + def toDataType(parquetType: ParquetType): DataType = { + if (parquetType.isPrimitive) toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) + else { + val groupType = parquetType.asGroupType() + parquetType.getOriginalType match { + case ParquetOriginalType.LIST | ParquetOriginalType.ENUM => { + val fields = groupType.getFields.map(toDataType(_)) + new ArrayType(fields.apply(0)) // array fields should have the same type + } + case _ => { // everything else nested becomes a Struct + val fields = groupType + .getFields + .map(ptype => new StructField( + ptype.getName, + toDataType(ptype), + ptype.getRepetition != Repetition.REQUIRED)) + new StructType(fields) + } + } + } + } + + def fromPrimitiveDataType(ctype: DataType): Option[ParquetPrimitiveTypeName] = ctype match { + case StringType => Some(ParquetPrimitiveTypeName.BINARY) + case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN) + case DoubleType => Some(ParquetPrimitiveTypeName.DOUBLE) + case ArrayType(ByteType) => Some(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) + case FloatType => Some(ParquetPrimitiveTypeName.FLOAT) + case IntegerType => Some(ParquetPrimitiveTypeName.INT32) + case LongType => Some(ParquetPrimitiveTypeName.INT64) + case _ => None + } + + def fromComplexDataType(ctype: DataType, name: String, nullable: Boolean = true): ParquetType = { + val repetition = + if (nullable) Repetition.OPTIONAL + else Repetition.REQUIRED + val primitiveType = fromPrimitiveDataType(ctype) + if (primitiveType.isDefined) { + new ParquetPrimitiveType(repetition, primitiveType.get, name) + } else { + ctype match { + case ArrayType(elementType: DataType) => { + val parquetElementType = fromComplexDataType(elementType, name + "_values", false) + new ParquetGroupType(repetition, name, parquetElementType) + } + case StructType(structFields) => { + val fields = structFields.map { + field => fromComplexDataType(field.dataType, field.name, false) + } + new ParquetGroupType(repetition, name, fields) + } + case _ => sys.error(s"Unsupported datatype $ctype") + } + } } def consumeType(consumer: RecordConsumer, ctype: DataType, record: Row, index: Int): Unit = { @@ -217,23 +264,18 @@ private[parquet] object ParquetTypesConverter { } } - def getSchema(schemaString : String) : MessageType = + def getSchema(schemaString: String) : MessageType = MessageTypeParser.parseMessageType(schemaString) - def convertToAttributes(parquetSchema: MessageType) : Seq[Attribute] = { - parquetSchema.getColumns.map { - case (desc) => - val ctype = toDataType(desc.getType) - val name: String = desc.getPath.mkString(".") - new AttributeReference(name, ctype, false)() - } + def convertToAttributes(parquetSchema: ParquetType): Seq[Attribute] = { + parquetSchema + .asGroupType() + .getFields + .map(field => new AttributeReference(field.getName, toDataType(field), field.getRepetition != Repetition.REQUIRED)()) } - // TODO: allow nesting? def convertFromAttributes(attributes: Seq[Attribute]): MessageType = { - val fields: Seq[ParquetType] = attributes.map { - a => new ParquetPrimitiveType(Repetition.OPTIONAL, fromDataType(a.dataType), a.name) - } + val fields = attributes.map(attribute => fromComplexDataType(attribute.dataType, attribute.name, attribute.nullable)) new MessageType("root", fields) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 46c7172985642..511cebfa945e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -17,14 +17,19 @@ package org.apache.spark.sql.parquet +import java.io.File + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.mapreduce.Job import parquet.example.data.{GroupWriter, Group} import parquet.example.data.simple.SimpleGroup -import parquet.hadoop.ParquetWriter +import parquet.hadoop.{ParquetReader, ParquetFileReader, ParquetWriter} import parquet.hadoop.api.WriteSupport import parquet.hadoop.api.WriteSupport.WriteContext +import parquet.hadoop.example.GroupReadSupport +import parquet.hadoop.util.ContextUtil import parquet.io.api.RecordConsumer import parquet.schema.{MessageType, MessageTypeParser} @@ -100,9 +105,39 @@ private[sql] object ParquetTestData { lazy val testData = new ParquetRelation(testDir.toURI.toString) + val testNestedSchema1 = + // from blogpost example, source: + // https://blog.twitter.com/2013/dremel-made-simple-with-parquet + // note: instead of string we have to use binary (?) otherwise + // Parquet gives us: + // IllegalArgumentException: expected one of [INT64, INT32, BOOLEAN, + // BINARY, FLOAT, DOUBLE, INT96, FIXED_LEN_BYTE_ARRAY] + """ + |message AddressBook { + |required binary owner; + |repeated binary ownerPhoneNumbers; + |repeated group contacts { + |required binary name; + |optional binary phoneNumber; + |} + |} + """.stripMargin + + val testNestedDir1 = Utils.createTempDir() + + lazy val testNestedData1 = new ParquetRelation(testNestedDir1.toURI.toString) + + // Implicit + // TODO: get rid of this since it is confusing! + implicit def makePath(dir: File): Path = { + new Path(new Path(dir.toURI), new Path("part-r-0.parquet")) + } + def writeFile() = { - testDir.delete + testDir.delete() val path: Path = new Path(new Path(testDir.toURI), new Path("part-r-0.parquet")) + val job = new Job() + val configuration: Configuration = ContextUtil.getConfiguration(job) val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) val writeSupport = new TestGroupWriteSupport(schema) val writer = new ParquetWriter[Group](path, writeSupport) @@ -150,5 +185,60 @@ private[sql] object ParquetTestData { } writer.close() } + + def writeNestedFile1() { + // example data from https://blog.twitter.com/2013/dremel-made-simple-with-parquet + testNestedDir1.delete() + val path: Path = testNestedDir1 + val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema1) + + val r1 = new SimpleGroup(schema) + r1.add(0, "Julien Le Dem") + r1.add(1, "555 123 4567") + r1.add(1, "555 666 1337") + r1.addGroup(2) + .append("name", "Dmitriy Ryaboy") + .append("phoneNumber", "555 987 6543") + r1.addGroup(2) + .append("name", "Chris Aniszczyk") + + val r2 = new SimpleGroup(schema) + r2.add(0, "A. Nonymous") + + // ParquetWriter initializes GroupWriteSupport with an empty configuration + // (it is after all not intended to be used in this way?) + // and members are private so we need to make our own + val writeSupport = new WriteSupport[Group] { + var groupWriter: GroupWriter = null + override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { + groupWriter = new GroupWriter(recordConsumer, schema) + } + override def init(configuration: Configuration): WriteContext = { + new WriteContext(schema, new java.util.HashMap[String, String]()) + } + override def write(record: Group) { + groupWriter.write(record) + } + } + val writer = new ParquetWriter[Group](path, writeSupport) + writer.write(r1) + writer.write(r2) + writer.close() + } + + def readNestedFile(): Unit = { + val configuration = new Configuration() + val fs: FileSystem = testNestedDir1.getFileSystem(configuration) + val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema1) + val outputStatus: FileStatus = fs.getFileStatus(testNestedDir1) + val footers = ParquetFileReader.readFooter(configuration, outputStatus) + val reader = new ParquetReader(testNestedDir1, new GroupReadSupport()) + val first = reader.read() + assert(first != null) + val second = reader.read() + assert(second != null) + assert(schema != null) + assert(footers != null) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 9810520bb9ae6..af197455d7509 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -65,6 +65,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA override def beforeAll() { ParquetTestData.writeFile() ParquetTestData.writeFilterFile() + ParquetTestData.writeNestedFile1() testRDD = parquetFile(ParquetTestData.testDir.toString) testRDD.registerAsTable("testsource") parquetFile(ParquetTestData.testFilterDir.toString) @@ -74,6 +75,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA override def afterAll() { Utils.deleteRecursively(ParquetTestData.testDir) Utils.deleteRecursively(ParquetTestData.testFilterDir) + Utils.deleteRecursively(ParquetTestData.testNestedDir1) // here we should also unregister the table?? } @@ -363,4 +365,39 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val query = sql(s"SELECT mystring FROM testfiltersource WHERE myint < 10") assert(query.collect().size === 10) } + + test("Importing nested File") { + ParquetTestData.readNestedFile() + val result = getRDD(ParquetTestData.testNestedData1).collect() + /*assert(result.size === 15) + result.zipWithIndex.foreach { + case (row, index) => { + val checkBoolean = + if (index % 3 == 0) + row(0) == true + else + row(0) == false + assert(checkBoolean === true, s"boolean field value in line $index did not match") + if (index % 5 == 0) assert(row(1) === 5, s"int field value in line $index did not match") + assert(row(2) === "abc", s"string field value in line $index did not match") + assert(row(3) === (index.toLong << 33), s"long value in line $index did not match") + assert(row(4) === 2.5F, s"float field value in line $index did not match") + assert(row(5) === 4.5D, s"double field value in line $index did not match") + } + }*/ + } + + /** + * Creates an empty SchemaRDD backed by a ParquetRelation. + * + * TODO: since this is so experimental it is better to have it here and not + * in SQLContext. Also note that when creating new AttributeReferences + * one needs to take care not to create duplicate Attribute ID's. + */ + private def createParquetFile(path: String, schema: (Tuple2[String, DataType])*): SchemaRDD = { + val attributes = schema.map(t => new AttributeReference(t._1, t._2)()) + new SchemaRDD( + TestSQLContext, + parquet.ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration)) + } } From 4d4892a2d5bfa2e1f91d05997b14cddf2c123822 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 27 Mar 2014 18:24:13 +0200 Subject: [PATCH 02/48] First commit nested Parquet read converters --- .../spark/sql/catalyst/types/dataTypes.scala | 10 +- .../spark/sql/parquet/ParquetRelation.scala | 62 +++-- .../sql/parquet/ParquetTableSupport.scala | 212 +++++++++++++++--- .../spark/sql/parquet/ParquetTestData.scala | 27 ++- .../spark/sql/parquet/ParquetQuerySuite.scala | 27 +-- 5 files changed, 258 insertions(+), 80 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index da34bd3a21503..3b1c695911bf5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -33,21 +33,23 @@ abstract class DataType { case object NullType extends DataType +trait PrimitiveType + abstract class NativeType extends DataType { type JvmType @transient val tag: TypeTag[JvmType] val ordering: Ordering[JvmType] } -case object StringType extends NativeType { +case object StringType extends NativeType with PrimitiveType { type JvmType = String @transient lazy val tag = typeTag[JvmType] val ordering = implicitly[Ordering[JvmType]] } -case object BinaryType extends DataType { +case object BinaryType extends DataType with PrimitiveType { type JvmType = Array[Byte] } -case object BooleanType extends NativeType { +case object BooleanType extends NativeType with PrimitiveType { type JvmType = Boolean @transient lazy val tag = typeTag[JvmType] val ordering = implicitly[Ordering[JvmType]] @@ -63,7 +65,7 @@ case object TimestampType extends NativeType { } } -abstract class NumericType extends NativeType { +abstract class NumericType extends NativeType with PrimitiveType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 90ce1d7076b4e..f07ab45c3b356 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -161,7 +161,7 @@ private[sql] object ParquetRelation { } if (fs.exists(path) && - !fs.getFileStatus(path) + !fs.getFileStatus(path) .getPermission .getUserAction .implies(FsAction.READ_WRITE)) { @@ -173,6 +173,8 @@ private[sql] object ParquetRelation { } private[parquet] object ParquetTypesConverter { + def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass + def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { // for now map binary to string type // TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema @@ -192,24 +194,36 @@ private[parquet] object ParquetTypesConverter { } def toDataType(parquetType: ParquetType): DataType = { - if (parquetType.isPrimitive) toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) + if (parquetType.isPrimitive) { + toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) + } else { val groupType = parquetType.asGroupType() parquetType.getOriginalType match { - case ParquetOriginalType.LIST | ParquetOriginalType.ENUM => { - val fields = groupType.getFields.map(toDataType(_)) + // if the schema was constructed programmatically there may be hints how to convert + // it inside the metadata via the OriginalType field + case ParquetOriginalType.LIST | ParquetOriginalType.ENUM => { // TODO: check enums! + val fields = groupType.getFields.map(toDataType(_)) new ArrayType(fields.apply(0)) // array fields should have the same type } - case _ => { // everything else nested becomes a Struct - val fields = groupType - .getFields - .map(ptype => new StructField( - ptype.getName, - toDataType(ptype), - ptype.getRepetition != Repetition.REQUIRED)) - new StructType(fields) + case _ => { // everything else nested becomes a Struct, unless it has a single repeated field + // in which case it becomes an array (this should correspond to the inverse operation of + // parquet.schema.ConversionPatterns.listType) + if (groupType.getFieldCount == 1 && groupType.getFields.apply(0).getRepetition == Repetition.REPEATED) { + val elementType = toDataType(groupType.getFields.apply(0)) + new ArrayType(elementType) + } else { + val fields = groupType + .getFields + .map(ptype => new StructField( + ptype.getName, + toDataType(ptype), + ptype.getRepetition != Repetition.REQUIRED)) + new StructType(fields) + } } } + //} } } @@ -224,24 +238,32 @@ private[parquet] object ParquetTypesConverter { case _ => None } - def fromComplexDataType(ctype: DataType, name: String, nullable: Boolean = true): ParquetType = { + def fromDataType(ctype: DataType, name: String, nullable: Boolean = true, inArray: Boolean = false): ParquetType = { val repetition = - if (nullable) Repetition.OPTIONAL - else Repetition.REQUIRED + if (inArray) Repetition.REPEATED + else { + if (nullable) Repetition.OPTIONAL + else Repetition.REQUIRED + } val primitiveType = fromPrimitiveDataType(ctype) if (primitiveType.isDefined) { new ParquetPrimitiveType(repetition, primitiveType.get, name) } else { ctype match { case ArrayType(elementType: DataType) => { - val parquetElementType = fromComplexDataType(elementType, name + "_values", false) - new ParquetGroupType(repetition, name, parquetElementType) + // TODO: "values" is a generic name but without it the Parquet column path would + // be incomplete and values may be silently dropped; better would be to give + // Array elements a name of some sort (and specify whether they are nullable), + // as in StructField + val parquetElementType = fromDataType(elementType, "values", nullable=false, inArray=true) + ConversionPatterns.listType(repetition, name, parquetElementType) } + // TODO: test structs inside arrays case StructType(structFields) => { val fields = structFields.map { - field => fromComplexDataType(field.dataType, field.name, false) + field => fromDataType(field.dataType, field.name, field.nullable) } - new ParquetGroupType(repetition, name, fields) + new ParquetGroupType(Repetition.REPEATED, name, fields) } case _ => sys.error(s"Unsupported datatype $ctype") } @@ -275,7 +297,7 @@ private[parquet] object ParquetTypesConverter { } def convertFromAttributes(attributes: Seq[Attribute]): MessageType = { - val fields = attributes.map(attribute => fromComplexDataType(attribute.dataType, attribute.name, attribute.nullable)) + val fields = attributes.map(attribute => fromDataType(attribute.dataType, attribute.name, attribute.nullable)) new MessageType("root", fields) } 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 71ba0fecce47a..f54d55e2462d7 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 @@ -29,6 +29,8 @@ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.parquet.ParquetRelation.RowType +import org.apache.spark.sql.parquet.CatalystConverter.FieldType /** * A `parquet.io.api.RecordMaterializer` for Rows. @@ -143,44 +145,129 @@ private[parquet] object RowWriteSupport { val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema" } -/** - * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record to a `Row` object. - * - * @param schema The corresponding Catalyst schema in the form of a list of attributes. - */ -private[parquet] class CatalystGroupConverter( - schema: Seq[Attribute], - protected[parquet] val current: ParquetRelation.RowType) extends GroupConverter { - - def this(schema: Seq[Attribute]) = this(schema, new ParquetRelation.RowType(schema.length)) - - val converters: Array[Converter] = schema.map { - a => a.dataType match { +private[parquet] object CatalystConverter { + type FieldType = StructField + + protected[parquet] def createConverter(field: FieldType, fieldIndex: Int, parent: CatalystConverter): Converter = { + val fieldType: DataType = field.dataType + fieldType match { + case ArrayType(elementType: DataType) => { + val numberOfFields = 10 // TODO: where do we get this info from? + val subArray = new ParquetRelation.RowType(numberOfFields) + parent.updateField(fieldIndex, subArray) + new CatalystArrayConverter(elementType, fieldIndex, parent, subArray) + } + case StructType(fields: Seq[StructField]) => { + val subArray = new ParquetRelation.RowType(fields.size) + parent.updateField(fieldIndex, subArray) + new CatalystGroupConverter(fields, fieldIndex, parent, subArray) // TODO: check if field type changes + } case ctype: NativeType => // note: for some reason matching for StringType fails so use this ugly if instead if (ctype == StringType) { - new CatalystPrimitiveStringConverter(this, schema.indexOf(a)) + new CatalystPrimitiveStringConverter(parent, fieldIndex) } else { - new CatalystPrimitiveConverter(this, schema.indexOf(a)) + new CatalystPrimitiveConverter(parent, fieldIndex) } case _ => throw new RuntimeException( - s"unable to convert datatype ${a.dataType.toString} in CatalystGroupConverter") + s"unable to convert datatype ${field.dataType.toString} in CatalystGroupConverter") } - }.toArray + } +} - override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) +trait CatalystConverter { + + // the number of fields this group has + val size: Int + + // the index of this converter in the parent + protected[parquet] val index: Int + + // the data (possibly nested) + protected[parquet] val current: ParquetRelation.RowType + + // the parent converter + protected[parquet] val parent: CatalystConverter + + // should be only called in root group converter! + def getCurrentRecord: ParquetRelation.RowType = current + + // for child converters to update upstream values + protected [parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + current.update(fieldIndex, value) + if (parent != null) parent.updateField(index, current) + } - private[parquet] def getCurrentRecord: ParquetRelation.RowType = current + protected [parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { + current.setBoolean(fieldIndex, value) + if (parent != null) parent.updateField(index, current) + } - override def start(): Unit = { + protected [parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { + current.setInt(fieldIndex, value) + if (parent != null) parent.updateField(index, current) + } + + protected [parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { + current.setLong(fieldIndex, value) + if (parent != null) parent.updateField(index, current) + } + + protected [parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { + current.setDouble(fieldIndex, value) + if (parent != null) parent.updateField(index, current) + } + + protected [parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { + current.setFloat(fieldIndex, value) + if (parent != null) parent.updateField(index, current) + } + + protected [parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { + current.update(fieldIndex, value.getBytes) + if (parent != null) parent.updateField(index, current) + } + + protected [parquet] def updateString(fieldIndex: Int, value: Binary): Unit = { + current.setString(fieldIndex, value.toStringUsingUTF8) + if (parent != null) parent.updateField(index, current) + } + + def start(): Unit = { var i = 0 - while (i < schema.length) { + while (i < size) { current.setNullAt(i) i = i + 1 } } - override def end(): Unit = {} + def end(): Unit = {} +} + +/** + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. + * + * @param schema The corresponding Catalyst schema in the form of a list of attributes. + */ +class CatalystGroupConverter( + private[parquet] val schema: Seq[FieldType], + protected[parquet] val index: Int, + protected[parquet] val parent: CatalystConverter, + protected[parquet] val current: RowType) extends GroupConverter with CatalystConverter { + + def this(schema: Seq[FieldType], index:Int, parent: CatalystConverter) = + this(schema, index, parent, new ParquetRelation.RowType(schema.length)) + + def this(attributes: Seq[Attribute]) = + this(attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), 0, null) + + protected [parquet] val converters: Array[Converter] = + schema.map(field => CatalystConverter.createConverter(field, schema.indexOf(field), this)).toArray + + override val size = schema.size + + override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) } /** @@ -190,26 +277,26 @@ private[parquet] class CatalystGroupConverter( * @param fieldIndex The index inside the record. */ private[parquet] class CatalystPrimitiveConverter( - parent: CatalystGroupConverter, + parent: CatalystConverter, fieldIndex: Int) extends PrimitiveConverter { // TODO: consider refactoring these together with ParquetTypesConverter override def addBinary(value: Binary): Unit = - parent.getCurrentRecord.update(fieldIndex, value.getBytes) + parent.updateBinary(fieldIndex, value) override def addBoolean(value: Boolean): Unit = - parent.getCurrentRecord.setBoolean(fieldIndex, value) + parent.updateBoolean(fieldIndex, value) override def addDouble(value: Double): Unit = - parent.getCurrentRecord.setDouble(fieldIndex, value) + parent.updateDouble(fieldIndex, value) override def addFloat(value: Float): Unit = - parent.getCurrentRecord.setFloat(fieldIndex, value) + parent.updateFloat(fieldIndex, value) override def addInt(value: Int): Unit = - parent.getCurrentRecord.setInt(fieldIndex, value) + parent.updateInt(fieldIndex, value) override def addLong(value: Long): Unit = - parent.getCurrentRecord.setLong(fieldIndex, value) + parent.updateLong(fieldIndex, value) } /** @@ -220,8 +307,71 @@ private[parquet] class CatalystPrimitiveConverter( * @param fieldIndex The index inside the record. */ private[parquet] class CatalystPrimitiveStringConverter( - parent: CatalystGroupConverter, + parent: CatalystConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { override def addBinary(value: Binary): Unit = - parent.getCurrentRecord.setString(fieldIndex, value.toStringUsingUTF8) + parent.updateString(fieldIndex, value) } + +class CatalystArrayConverter( + val elementType: DataType, + val size: Int, + val index: Int, + protected[parquet] val parent: CatalystConverter, + protected[parquet] val current: ParquetRelation.RowType) + extends GroupConverter with CatalystConverter { + + private var arrayOffset: Int = 0 + + def this(elementType: DataType, index: Int, parent: CatalystConverter, current: ParquetRelation.RowType) = + this(elementType, current.size, index, parent, current) + + def this(elementType: DataType, size: Int, index: Int, parent: CatalystConverter) = + this(elementType, size, index, parent, new ParquetRelation.RowType(size)) + + protected[parquet] val converter: Converter = CatalystConverter.createConverter( + new CatalystConverter.FieldType("values", elementType, false), 0, this) + + override def getConverter(fieldIndex: Int): Converter = converter + + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + super.updateField(arrayOffset + fieldIndex, value) + arrayOffset = arrayOffset + 1 + } + + override protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit = { + super.updateString(arrayOffset + fieldIndex, value) + arrayOffset = arrayOffset + 1 + } + + override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { + super.updateBoolean(arrayOffset + fieldIndex, value) + arrayOffset = arrayOffset + 1 + } + + override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { + super.updateInt(arrayOffset + fieldIndex, value) + arrayOffset = arrayOffset + 1 + } + + override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { + super.updateLong(arrayOffset + fieldIndex, value) + arrayOffset = arrayOffset + 1 + } + + override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { + super.updateDouble(arrayOffset + fieldIndex, value) + arrayOffset = arrayOffset + 1 + } + + override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { + super.updateFloat(arrayOffset + fieldIndex, value) + arrayOffset = arrayOffset + 1 + } + + override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { + super.updateBinary(arrayOffset + fieldIndex, value) + arrayOffset = arrayOffset + 1 + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 511cebfa945e2..b9ce6c07bef4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -106,20 +106,27 @@ private[sql] object ParquetTestData { lazy val testData = new ParquetRelation(testDir.toURI.toString) val testNestedSchema1 = - // from blogpost example, source: + // based on blogpost example, source: // https://blog.twitter.com/2013/dremel-made-simple-with-parquet // note: instead of string we have to use binary (?) otherwise // Parquet gives us: // IllegalArgumentException: expected one of [INT64, INT32, BOOLEAN, // BINARY, FLOAT, DOUBLE, INT96, FIXED_LEN_BYTE_ARRAY] + // Also repeated primitives seem tricky to convert (AvroParquet + // only uses them in arrays?) so only use at most one in each group + // and nothing else in that group (-> is mapped to array)! + // The "values" inside ownerPhoneNumbers is a keyword currently + // so that array types can be translated correctly. """ |message AddressBook { - |required binary owner; - |repeated binary ownerPhoneNumbers; - |repeated group contacts { - |required binary name; - |optional binary phoneNumber; - |} + |required binary owner; + |optional group ownerPhoneNumbers { + |repeated binary values; + |} + |repeated group contacts { + |required binary name; + |optional binary phoneNumber; + |} |} """.stripMargin @@ -194,8 +201,10 @@ private[sql] object ParquetTestData { val r1 = new SimpleGroup(schema) r1.add(0, "Julien Le Dem") - r1.add(1, "555 123 4567") - r1.add(1, "555 666 1337") + r1.addGroup(1) + .append("values", "555 123 4567") + .append("values", "555 666 1337") + .append("values", "XXX XXX XXXX") r1.addGroup(2) .append("name", "Dmitriy Ryaboy") .append("phoneNumber", "555 987 6543") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index af197455d7509..66277a031dc9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -369,22 +369,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Importing nested File") { ParquetTestData.readNestedFile() val result = getRDD(ParquetTestData.testNestedData1).collect() - /*assert(result.size === 15) - result.zipWithIndex.foreach { - case (row, index) => { - val checkBoolean = - if (index % 3 == 0) - row(0) == true - else - row(0) == false - assert(checkBoolean === true, s"boolean field value in line $index did not match") - if (index % 5 == 0) assert(row(1) === 5, s"int field value in line $index did not match") - assert(row(2) === "abc", s"string field value in line $index did not match") - assert(row(3) === (index.toLong << 33), s"long value in line $index did not match") - assert(row(4) === 2.5F, s"float field value in line $index did not match") - assert(row(5) === 4.5D, s"double field value in line $index did not match") - } - }*/ + assert(result != null) + assert(result.size === 2) + assert(result(0).size === 3) + assert(result(1).apply(1) === null) + assert(result(1).apply(2) === null) + assert(result(1).apply(0) === "A. Nonymous") + assert(result(0).apply(0).isInstanceOf[java.lang.String]) + assert(result(0).apply(0) === "Julien Le Dem") + assert(result(0).apply(1).asInstanceOf[Row].apply(0) === "555 123 4567") + assert(result(0).apply(1).asInstanceOf[Row].apply(2) === "XXX XXX XXXX") + assert(result(0).apply(1).asInstanceOf[Row].apply(3) === null) // this should not even be there! } /** From 6125c75f9dcf4429730651c0d033a3b8245b6079 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 27 Mar 2014 19:48:16 +0200 Subject: [PATCH 03/48] First working nested Parquet record input --- .../spark/sql/parquet/ParquetRelation.scala | 37 +++- .../sql/parquet/ParquetTableSupport.scala | 199 +++++++++--------- .../spark/sql/parquet/ParquetTestData.scala | 2 + 3 files changed, 124 insertions(+), 114 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index f07ab45c3b356..73225a0ddcb46 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -202,9 +202,12 @@ private[parquet] object ParquetTypesConverter { parquetType.getOriginalType match { // if the schema was constructed programmatically there may be hints how to convert // it inside the metadata via the OriginalType field - case ParquetOriginalType.LIST | ParquetOriginalType.ENUM => { // TODO: check enums! - val fields = groupType.getFields.map(toDataType(_)) - new ArrayType(fields.apply(0)) // array fields should have the same type + case ParquetOriginalType.LIST => { // TODO: check enums! + val fields = groupType.getFields.map { + field => new StructField(field.getName, toDataType(field), field.getRepetition != Repetition.REQUIRED) + } + if (fields.size == 1) new ArrayType(fields.apply(0).dataType) + new ArrayType(StructType(fields)) } case _ => { // everything else nested becomes a Struct, unless it has a single repeated field // in which case it becomes an array (this should correspond to the inverse operation of @@ -219,11 +222,14 @@ private[parquet] object ParquetTypesConverter { ptype.getName, toDataType(ptype), ptype.getRepetition != Repetition.REQUIRED)) - new StructType(fields) + if (groupType.getFieldCount == 1) { // single field, either optional or required + new StructType(fields) + } else { // multi field repeated group, which we map into an array of structs + new ArrayType(StructType(fields)) + } } } } - //} } } @@ -251,12 +257,21 @@ private[parquet] object ParquetTypesConverter { } else { ctype match { case ArrayType(elementType: DataType) => { - // TODO: "values" is a generic name but without it the Parquet column path would - // be incomplete and values may be silently dropped; better would be to give - // Array elements a name of some sort (and specify whether they are nullable), - // as in StructField - val parquetElementType = fromDataType(elementType, "values", nullable=false, inArray=true) - ConversionPatterns.listType(repetition, name, parquetElementType) + elementType match { + case StructType(fields) => { // first case: array of structs + val parquetFieldTypes = fields.map(f => fromDataType(f.dataType, f.name, f.nullable, false)) + new ParquetGroupType(Repetition.REPEATED, name, ParquetOriginalType.LIST, parquetFieldTypes) + //ConversionPatterns.listType(Repetition.REPEATED, name, parquetFieldTypes) + } + case _ => { // second case: array of primitive types + // TODO: "values" is a generic name but without it the Parquet column path would + // be incomplete and values may be silently dropped; better would be to give + // Array elements a name of some sort (and specify whether they are nullable), + // as in StructField + val parquetElementType = fromDataType(elementType, "values", nullable=false, inArray=true) + ConversionPatterns.listType(repetition, name, parquetElementType) + } + } } // TODO: test structs inside arrays case StructType(structFields) => { 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 f54d55e2462d7..63d5a26b5d8d3 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,6 +17,8 @@ package org.apache.spark.sql.parquet +import collection.mutable.{ArrayBuffer, Buffer} + import org.apache.hadoop.conf.Configuration import parquet.column.ParquetProperties @@ -27,10 +29,11 @@ import parquet.io.api._ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} +import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, Row} import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.parquet.ParquetRelation.RowType import org.apache.spark.sql.parquet.CatalystConverter.FieldType +import org.apache.spark.sql.parquet.ParquetRelation.RowType +import scala.collection.mutable /** * A `parquet.io.api.RecordMaterializer` for Rows. @@ -152,16 +155,15 @@ private[parquet] object CatalystConverter { val fieldType: DataType = field.dataType fieldType match { case ArrayType(elementType: DataType) => { - val numberOfFields = 10 // TODO: where do we get this info from? - val subArray = new ParquetRelation.RowType(numberOfFields) - parent.updateField(fieldIndex, subArray) - new CatalystArrayConverter(elementType, fieldIndex, parent, subArray) - } - case StructType(fields: Seq[StructField]) => { - val subArray = new ParquetRelation.RowType(fields.size) - parent.updateField(fieldIndex, subArray) - new CatalystGroupConverter(fields, fieldIndex, parent, subArray) // TODO: check if field type changes + elementType match { + case StructType(fields) => + if (fields.size > 1) new CatalystGroupConverter(fields, fieldIndex, parent) //CatalystStructArrayConverter(fields, fieldIndex, parent) + else new CatalystArrayConverter(fields(0).dataType, fieldIndex, parent) + case _ => new CatalystArrayConverter(elementType, fieldIndex, parent) + } } + case StructType(fields: Seq[StructField]) => + new CatalystGroupConverter(fields, fieldIndex, parent) case ctype: NativeType => // note: for some reason matching for StringType fails so use this ugly if instead if (ctype == StringType) { @@ -178,70 +180,42 @@ private[parquet] object CatalystConverter { trait CatalystConverter { // the number of fields this group has - val size: Int + protected[parquet] val size: Int // the index of this converter in the parent protected[parquet] val index: Int - // the data (possibly nested) - protected[parquet] val current: ParquetRelation.RowType - // the parent converter protected[parquet] val parent: CatalystConverter - // should be only called in root group converter! - def getCurrentRecord: ParquetRelation.RowType = current - // for child converters to update upstream values - protected [parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - current.update(fieldIndex, value) - if (parent != null) parent.updateField(index, current) - } + protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit - protected [parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { - current.setBoolean(fieldIndex, value) - if (parent != null) parent.updateField(index, current) - } + // TODO: in the future consider using specific methods to avoid autoboxing + protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = + updateField(fieldIndex, value) - protected [parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { - current.setInt(fieldIndex, value) - if (parent != null) parent.updateField(index, current) - } + protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = + updateField(fieldIndex, value) - protected [parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { - current.setLong(fieldIndex, value) - if (parent != null) parent.updateField(index, current) - } + protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = + updateField(fieldIndex, value) - protected [parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { - current.setDouble(fieldIndex, value) - if (parent != null) parent.updateField(index, current) - } + protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = + updateField(fieldIndex, value) - protected [parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { - current.setFloat(fieldIndex, value) - if (parent != null) parent.updateField(index, current) - } + protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = + updateField(fieldIndex, value) - protected [parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { - current.update(fieldIndex, value.getBytes) - if (parent != null) parent.updateField(index, current) - } + protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = + updateField(fieldIndex, value.getBytes) - protected [parquet] def updateString(fieldIndex: Int, value: Binary): Unit = { - current.setString(fieldIndex, value.toStringUsingUTF8) - if (parent != null) parent.updateField(index, current) - } + protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit = + updateField(fieldIndex, value.toStringUsingUTF8) - def start(): Unit = { - var i = 0 - while (i < size) { - current.setNullAt(i) - i = i + 1 - } - } + protected[parquet] def isRootConverter: Boolean = parent == null - def end(): Unit = {} + protected[parquet] def clearBuffer(): Unit } /** @@ -254,11 +228,13 @@ class CatalystGroupConverter( private[parquet] val schema: Seq[FieldType], protected[parquet] val index: Int, protected[parquet] val parent: CatalystConverter, - protected[parquet] val current: RowType) extends GroupConverter with CatalystConverter { + protected[parquet] var current: ArrayBuffer[Any], + protected[parquet] var buffer: ArrayBuffer[ArrayBuffer[Any]]) extends GroupConverter with CatalystConverter { - def this(schema: Seq[FieldType], index:Int, parent: CatalystConverter) = - this(schema, index, parent, new ParquetRelation.RowType(schema.length)) + def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) = + this(schema, index, parent, current=null, buffer=new ArrayBuffer[ArrayBuffer[Any]](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) + // This constructor is used for the root converter only def this(attributes: Seq[Attribute]) = this(attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), 0, null) @@ -267,7 +243,41 @@ class CatalystGroupConverter( override val size = schema.size + // Should be only called in root group converter! + def getCurrentRecord: Row = new GenericRow { + override val values: Array[Any] = 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 = { + // TODO: reuse buffer? + buffer = new ArrayBuffer[ArrayBuffer[Any]](CatalystArrayConverter.INITIAL_ARRAY_SIZE) + } + + override def start(): Unit = { + // TODO: reuse buffer? + // Allocate new array in the root converter (others will be called clearBuffer() on) + current = ArrayBuffer.fill(schema.length)(null) + converters.foreach { + converter => if (!converter.isPrimitive) { + converter.asInstanceOf[CatalystConverter].clearBuffer + } + } + } + + // TODO: think about reusing the buffer + override def end(): Unit = { + if (!isRootConverter) { + assert(current!=null) // there should be no empty groups + buffer.append(current) + parent.updateField(index, buffer) + } + } } /** @@ -313,65 +323,48 @@ private[parquet] class CatalystPrimitiveStringConverter( parent.updateString(fieldIndex, value) } +object CatalystArrayConverter { + val INITIAL_ARRAY_SIZE = 20 +} + +// this is for single-element groups of primitive or complex types +// Note: AvroParquet only uses arrays for primitive types (?) class CatalystArrayConverter( val elementType: DataType, - val size: Int, val index: Int, protected[parquet] val parent: CatalystConverter, - protected[parquet] val current: ParquetRelation.RowType) + protected[parquet] var buffer: Buffer[Any]) extends GroupConverter with CatalystConverter { + // TODO: In the future consider using native arrays instead of buffer for primitive types for + // performance reasons (autoboxing) - private var arrayOffset: Int = 0 - - def this(elementType: DataType, index: Int, parent: CatalystConverter, current: ParquetRelation.RowType) = - this(elementType, current.size, index, parent, current) - - def this(elementType: DataType, size: Int, index: Int, parent: CatalystConverter) = - this(elementType, size, index, parent, new ParquetRelation.RowType(size)) + 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("values", elementType, false), 0, this) + new CatalystConverter.FieldType("values", elementType, false), fieldIndex=0, parent=this) override def getConverter(fieldIndex: Int): Converter = converter - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - super.updateField(arrayOffset + fieldIndex, value) - arrayOffset = arrayOffset + 1 - } - - override protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit = { - super.updateString(arrayOffset + fieldIndex, value) - arrayOffset = arrayOffset + 1 - } + override val size = 1 // arrays have only one (repeated) field, which is its elements - override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { - super.updateBoolean(arrayOffset + fieldIndex, value) - arrayOffset = arrayOffset + 1 - } + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = buffer += value - override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { - super.updateInt(arrayOffset + fieldIndex, value) - arrayOffset = arrayOffset + 1 + override protected[parquet] def clearBuffer(): Unit = { + // TODO: reuse buffer? + buffer = new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE) } - override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { - super.updateLong(arrayOffset + fieldIndex, value) - arrayOffset = arrayOffset + 1 - } - - override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { - super.updateDouble(arrayOffset + fieldIndex, value) - arrayOffset = arrayOffset + 1 - } - - override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { - super.updateFloat(arrayOffset + fieldIndex, value) - arrayOffset = arrayOffset + 1 + override def start(): Unit = { + if (!converter.isPrimitive) { + converter.asInstanceOf[CatalystConverter].clearBuffer + } } - override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { - super.updateBinary(arrayOffset + fieldIndex, value) - arrayOffset = arrayOffset + 1 + // TODO: think about reusing the buffer + override def end(): Unit = { + if (parent != null) parent.updateField(index, buffer) } } +// TODO: add MapConverter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index b9ce6c07bef4e..1242b530f3329 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -206,9 +206,11 @@ private[sql] object ParquetTestData { .append("values", "555 666 1337") .append("values", "XXX XXX XXXX") r1.addGroup(2) + // .addGroup(0) .append("name", "Dmitriy Ryaboy") .append("phoneNumber", "555 987 6543") r1.addGroup(2) + // .addGroup(0) .append("name", "Chris Aniszczyk") val r2 = new SimpleGroup(schema) From 745a42b52407583db6f65c3898d59066ec751acf Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Tue, 1 Apr 2014 16:17:02 +0300 Subject: [PATCH 04/48] Completing testcase for nested data (Addressbook( --- .../spark/sql/parquet/ParquetQuerySuite.scala | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 66277a031dc9a..5162aed92bf6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -28,12 +28,10 @@ import parquet.schema.MessageTypeParser import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util.getTempFilePath -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.TestData import org.apache.spark.sql.SchemaRDD -import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.expressions.Equals +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.util.Utils @@ -366,20 +364,26 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(query.collect().size === 10) } - test("Importing nested File") { + test("Importing nested Parquet file (Addressbook)") { ParquetTestData.readNestedFile() val result = getRDD(ParquetTestData.testNestedData1).collect() assert(result != null) assert(result.size === 2) - assert(result(0).size === 3) - assert(result(1).apply(1) === null) - assert(result(1).apply(2) === null) - assert(result(1).apply(0) === "A. Nonymous") - assert(result(0).apply(0).isInstanceOf[java.lang.String]) - assert(result(0).apply(0) === "Julien Le Dem") - assert(result(0).apply(1).asInstanceOf[Row].apply(0) === "555 123 4567") - assert(result(0).apply(1).asInstanceOf[Row].apply(2) === "XXX XXX XXXX") - assert(result(0).apply(1).asInstanceOf[Row].apply(3) === null) // this should not even be there! + val first_record = result(0) + val second_record = result(1) + val first_owner_numbers = result(0).apply(1).asInstanceOf[ArrayBuffer[Any]] + val first_contacts = result(0).apply(2).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]] + assert(first_record.size === 3) + assert(second_record.apply(1) === null) + assert(second_record.apply(2) === null) + assert(second_record.apply(0) === "A. Nonymous") + assert(first_record.apply(0) === "Julien Le Dem") + assert(first_owner_numbers.apply(0) === "555 123 4567") + assert(first_owner_numbers.apply(2) === "XXX XXX XXXX") + assert(first_contacts.apply(0).size === 2) + assert(first_contacts.apply(0).apply(0) === "Dmitriy Ryaboy") + assert(first_contacts.apply(0).apply(1) === "555 987 6543") + assert(first_contacts.apply(1).apply(0) === "Chris Aniszczyk") } /** From ddb40d28f54c5a030b75fbad72546982ddcf38fc Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Tue, 1 Apr 2014 17:15:23 +0300 Subject: [PATCH 05/48] Extending tests for nested Parquet data --- .../spark/sql/parquet/ParquetTestData.scala | 83 +++++++++++++------ .../spark/sql/parquet/ParquetQuerySuite.scala | 15 +++- 2 files changed, 72 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 1242b530f3329..16725a07b295e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -130,9 +130,34 @@ private[sql] object ParquetTestData { |} """.stripMargin + + val testNestedSchema2 = + """ + |message TestNested2 { + |required int32 firstInt; + |optional int32 secondInt; + |optional group longs { + |repeated int64 values; + |} + |required group booleanNumberPairs { + |required double value; + |optional boolean truth; + |} + |required group outerouter { + |required group outer { + |required group inner { + |required int32 number; + |} + |} + |} + |} + """.stripMargin + val testNestedDir1 = Utils.createTempDir() + val testNestedDir2 = Utils.createTempDir() lazy val testNestedData1 = new ParquetRelation(testNestedDir1.toURI.toString) + lazy val testNestedData2 = new ParquetRelation(testNestedDir2.toURI.toString) // Implicit // TODO: get rid of this since it is confusing! @@ -216,40 +241,50 @@ private[sql] object ParquetTestData { val r2 = new SimpleGroup(schema) r2.add(0, "A. Nonymous") - // ParquetWriter initializes GroupWriteSupport with an empty configuration - // (it is after all not intended to be used in this way?) - // and members are private so we need to make our own - val writeSupport = new WriteSupport[Group] { - var groupWriter: GroupWriter = null - override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { - groupWriter = new GroupWriter(recordConsumer, schema) - } - override def init(configuration: Configuration): WriteContext = { - new WriteContext(schema, new java.util.HashMap[String, String]()) - } - override def write(record: Group) { - groupWriter.write(record) - } - } + val writeSupport = new TestGroupWriteSupport(schema) val writer = new ParquetWriter[Group](path, writeSupport) writer.write(r1) writer.write(r2) writer.close() } - def readNestedFile(): Unit = { + def writeNestedFile2() { + testNestedDir2.delete() + val path: Path = testNestedDir2 + val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema2) + + val r1 = new SimpleGroup(schema) + r1.add(0, 1) + r1.add(1, 7) + val longs = r1.addGroup(2) + longs.add("values", 1.toLong << 32) + longs.add("values", 1.toLong << 33) + longs.add("values", 1.toLong << 34) + val booleanNumberPairs = r1.addGroup(3) + booleanNumberPairs.add("value", 2.5) + booleanNumberPairs.add("truth", false) + r1.addGroup(4).addGroup(0).addGroup(0).add("number", 7) + r1.addGroup(4).addGroup(0).addGroup(0).add("number", 8) + r1.addGroup(4).addGroup(0).addGroup(0).add("number", 9) + + val writeSupport = new TestGroupWriteSupport(schema) + val writer = new ParquetWriter[Group](path, writeSupport) + writer.write(r1) + writer.close() + } + + + def readNestedFile(path: File, schemaString: String): Unit = { val configuration = new Configuration() - val fs: FileSystem = testNestedDir1.getFileSystem(configuration) - val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema1) - val outputStatus: FileStatus = fs.getFileStatus(testNestedDir1) + val fs: FileSystem = path.getFileSystem(configuration) + val schema: MessageType = MessageTypeParser.parseMessageType(schemaString) + assert(schema != null) + val outputStatus: FileStatus = fs.getFileStatus(path) val footers = ParquetFileReader.readFooter(configuration, outputStatus) - val reader = new ParquetReader(testNestedDir1, new GroupReadSupport()) + assert(footers != null) + val reader = new ParquetReader(path, new GroupReadSupport()) val first = reader.read() assert(first != null) - val second = reader.read() - assert(second != null) - assert(schema != null) - assert(footers != null) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 5162aed92bf6d..9f2e6227c778c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -64,6 +64,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA ParquetTestData.writeFile() ParquetTestData.writeFilterFile() ParquetTestData.writeNestedFile1() + ParquetTestData.writeNestedFile2() testRDD = parquetFile(ParquetTestData.testDir.toString) testRDD.registerAsTable("testsource") parquetFile(ParquetTestData.testFilterDir.toString) @@ -74,6 +75,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(ParquetTestData.testDir) Utils.deleteRecursively(ParquetTestData.testFilterDir) Utils.deleteRecursively(ParquetTestData.testNestedDir1) + Utils.deleteRecursively(ParquetTestData.testNestedDir2) // here we should also unregister the table?? } @@ -197,7 +199,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(rdd_copy(i).apply(1) === rdd_orig(i).value, s"value in line $i") } Utils.deleteRecursively(file) - assert(true) } test("insert (appending) to same table via Scala API") { @@ -365,7 +366,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Importing nested Parquet file (Addressbook)") { - ParquetTestData.readNestedFile() + ParquetTestData.readNestedFile( + ParquetTestData.testNestedFile1, + ParquetTestData.testNestedSchema1) val result = getRDD(ParquetTestData.testNestedData1).collect() assert(result != null) assert(result.size === 2) @@ -386,6 +389,14 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(first_contacts.apply(1).apply(0) === "Chris Aniszczyk") } + test("Importing nested Parquet file (nested numbers)") { + ParquetTestData.readNestedFile( + ParquetTestData.testNestedFile2, + ParquetTestData.testNestedSchema2) + val result = getRDD(ParquetTestData.testNestedData2).collect() + assert(result != null) + } + /** * Creates an empty SchemaRDD backed by a ParquetRelation. * From 1b1b3d698fd5ec294e82777723ce4916c7146a77 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 2 Apr 2014 15:42:19 +0300 Subject: [PATCH 06/48] Fixing one problem with nested arrays --- .../apache/spark/sql/parquet/ParquetRelation.scala | 9 ++++++--- .../spark/sql/parquet/ParquetTableSupport.scala | 9 ++++++--- .../apache/spark/sql/parquet/ParquetTestData.scala | 14 +++++++------- 3 files changed, 19 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 73225a0ddcb46..2ce262cb8504e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -206,8 +206,11 @@ private[parquet] object ParquetTypesConverter { val fields = groupType.getFields.map { field => new StructField(field.getName, toDataType(field), field.getRepetition != Repetition.REQUIRED) } - if (fields.size == 1) new ArrayType(fields.apply(0).dataType) - new ArrayType(StructType(fields)) + if (fields.size == 1) { + new ArrayType(fields.apply(0).dataType) + } else { + new ArrayType(StructType(fields)) + } } case _ => { // everything else nested becomes a Struct, unless it has a single repeated field // in which case it becomes an array (this should correspond to the inverse operation of @@ -260,7 +263,7 @@ private[parquet] object ParquetTypesConverter { elementType match { case StructType(fields) => { // first case: array of structs val parquetFieldTypes = fields.map(f => fromDataType(f.dataType, f.name, f.nullable, false)) - new ParquetGroupType(Repetition.REPEATED, name, ParquetOriginalType.LIST, parquetFieldTypes) + new ParquetGroupType(repetition, name, ParquetOriginalType.LIST, parquetFieldTypes) //ConversionPatterns.listType(Repetition.REPEATED, name, parquetFieldTypes) } case _ => { // second case: array of primitive types 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 63d5a26b5d8d3..cccdfd99aacc5 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 @@ -157,7 +157,7 @@ private[parquet] object CatalystConverter { case ArrayType(elementType: DataType) => { elementType match { case StructType(fields) => - if (fields.size > 1) new CatalystGroupConverter(fields, fieldIndex, parent) //CatalystStructArrayConverter(fields, fieldIndex, parent) + if (fields.size > 1) new CatalystGroupConverter(fields, fieldIndex, parent) else new CatalystArrayConverter(fields(0).dataType, fieldIndex, parent) case _ => new CatalystArrayConverter(elementType, fieldIndex, parent) } @@ -244,8 +244,11 @@ class CatalystGroupConverter( override val size = schema.size // Should be only called in root group converter! - def getCurrentRecord: Row = new GenericRow { - override val values: Array[Any] = current.toArray + def getCurrentRecord: Row = { + assert(isRootConverter, "getCurrentRecord should only be called in root group converter!") + new GenericRow { + override val values: Array[Any] = current.toArray + } } override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 16725a07b295e..0a527630c7052 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -143,10 +143,10 @@ private[sql] object ParquetTestData { |required double value; |optional boolean truth; |} - |required group outerouter { - |required group outer { - |required group inner { - |required int32 number; + |optional group outerouter { + |repeated group values { + |repeated group values { + |repeated int32 values; |} |} |} @@ -263,9 +263,9 @@ private[sql] object ParquetTestData { val booleanNumberPairs = r1.addGroup(3) booleanNumberPairs.add("value", 2.5) booleanNumberPairs.add("truth", false) - r1.addGroup(4).addGroup(0).addGroup(0).add("number", 7) - r1.addGroup(4).addGroup(0).addGroup(0).add("number", 8) - r1.addGroup(4).addGroup(0).addGroup(0).add("number", 9) + r1.addGroup(4).addGroup(0).addGroup(0).add("values", 7) + r1.addGroup(4).addGroup(0).addGroup(0).add("values", 8) + r1.addGroup(4).addGroup(0).addGroup(0).add("values", 9) val writeSupport = new TestGroupWriteSupport(schema) val writer = new ParquetWriter[Group](path, writeSupport) From 5d804614e54bd75cf560a43442fcdb7dcd563431 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 2 Apr 2014 17:00:46 +0300 Subject: [PATCH 07/48] fixing one problem with nested structs and breaking up files --- .../spark/sql/parquet/ParquetConverter.scala | 282 ++++++++++++++++++ .../spark/sql/parquet/ParquetRelation.scala | 42 ++- .../spark/sql/parquet/ParquetTestData.scala | 18 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 17 +- 4 files changed, 341 insertions(+), 18 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala 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 new file mode 100644 index 0000000000000..66c9c674fa657 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.parquet + +import scala.collection.mutable.{Buffer, ArrayBuffer} + +import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} +import org.apache.spark.sql.parquet.CatalystConverter.FieldType + +object CatalystConverter { + // The type internally used for fields + type FieldType = StructField + + // Note: repeated primitive fields that form an array (together with + // their surrounding group) need to have this name in the schema + // TODO: "values" is a generic name but without it the Parquet column path would + // be incomplete and values may be silently dropped; better would be to give + // primitive-type array elements a name of some sort + val ARRAY_ELEMENTS_SCHEMA_NAME = "values" + + protected[parquet] def createConverter( + field: FieldType, + fieldIndex: Int, + parent: CatalystConverter): Converter = { + val fieldType: DataType = field.dataType + fieldType match { + case ArrayType(elementType: DataType) => { + elementType match { + case StructType(fields) => + if (fields.size > 1) new CatalystGroupConverter(fields, fieldIndex, parent) + else new CatalystArrayConverter(fields(0).dataType, fieldIndex, parent) + case _ => new CatalystArrayConverter(elementType, fieldIndex, parent) + } + } + case StructType(fields: Seq[StructField]) => + new CatalystGroupConverter(fields, fieldIndex, parent) + case ctype: NativeType => + // note: for some reason matching for StringType fails so use this ugly if instead + if (ctype == StringType) new CatalystPrimitiveStringConverter(parent, fieldIndex) + else new CatalystPrimitiveConverter(parent, fieldIndex) + case _ => throw new RuntimeException( + s"unable to convert datatype ${field.dataType.toString} in CatalystGroupConverter") + } + } +} + +trait CatalystConverter { + // 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 + + // for child converters to update upstream values + protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit + + // TODO: in the future consider using specific methods to avoid autoboxing + 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 updateLong(fieldIndex: Int, value: Long): 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: Binary): Unit = + updateField(fieldIndex, value.toStringUsingUTF8) + + protected[parquet] def isRootConverter: Boolean = parent == null + + protected[parquet] def clearBuffer(): Unit +} + +/** + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. + * + * @param schema The corresponding Catalyst schema in the form of a list of attributes. + */ +class CatalystGroupConverter( + private[parquet] val schema: Seq[FieldType], + protected[parquet] val index: Int, + protected[parquet] val parent: CatalystConverter, + protected[parquet] var current: ArrayBuffer[Any], + protected[parquet] var buffer: ArrayBuffer[ArrayBuffer[Any]]) + extends GroupConverter with CatalystConverter { + + def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) = + this( + schema, + index, + parent, + current=null, + buffer=new ArrayBuffer[ArrayBuffer[Any]]( + CatalystArrayConverter.INITIAL_ARRAY_SIZE)) + + // This constructor is used for the root converter only + def this(attributes: Seq[Attribute]) = + this(attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), 0, null) + + protected [parquet] val converters: Array[Converter] = + schema.map(field => + CatalystConverter.createConverter(field, schema.indexOf(field), this)) + .toArray + + override val size = schema.size + + // Should be only called in root group converter! + def getCurrentRecord: Row = { + assert(isRootConverter, "getCurrentRecord should only be called in root group converter!") + new GenericRow { + override val values: Array[Any] = 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 = { + // TODO: reuse buffer? + buffer = new ArrayBuffer[ArrayBuffer[Any]](CatalystArrayConverter.INITIAL_ARRAY_SIZE) + } + + override def start(): Unit = { + // TODO: reuse buffer? + // Allocate new array in the root converter (others will be called clearBuffer() on) + current = ArrayBuffer.fill(schema.length)(null) + converters.foreach { + converter => if (!converter.isPrimitive) { + converter.asInstanceOf[CatalystConverter].clearBuffer + } + } + } + + // TODO: think about reusing the buffer + override def end(): Unit = { + if (!isRootConverter) { + assert(current!=null) // there should be no empty groups + buffer.append(current) + parent.updateField(index, buffer) + } + } +} + +/** + * 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. + */ +class CatalystPrimitiveConverter( + parent: CatalystConverter, + fieldIndex: Int) extends PrimitiveConverter { + // TODO: consider refactoring these together with ParquetTypesConverter + 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 strings (fixed-length byte arrays) + * into Catalyst Strings. + * + * @param parent The parent group converter. + * @param fieldIndex The index inside the record. + */ +class CatalystPrimitiveStringConverter( + parent: CatalystConverter, + fieldIndex: Int) + extends CatalystPrimitiveConverter(parent, fieldIndex) { + override def addBinary(value: Binary): Unit = + parent.updateString(fieldIndex, value) +} + +object CatalystArrayConverter { + val INITIAL_ARRAY_SIZE = 20 +} + +// this is for single-element groups of primitive or complex types +// Note: AvroParquet only uses arrays for primitive types (?) +class CatalystArrayConverter( + val elementType: DataType, + val index: Int, + protected[parquet] val parent: CatalystConverter, + protected[parquet] var buffer: Buffer[Any]) + extends GroupConverter with CatalystConverter { + // TODO: In the future consider using native arrays instead of buffer for + // primitive types for performance reasons + + 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 ={ + buffer += value + } + + override protected[parquet] def clearBuffer(): Unit = { + // TODO: reuse buffer? + buffer = new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE) + } + + override def start(): Unit = { + if (!converter.isPrimitive) { + converter.asInstanceOf[CatalystConverter].clearBuffer + } + } + + // TODO: think about reusing the buffer + override def end(): Unit = { + if (parent != null) parent.updateField(index, buffer) + } +} + +// TODO: add MapConverter + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 2ce262cb8504e..ce5e6d6f8775e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -228,7 +228,11 @@ private[parquet] object ParquetTypesConverter { if (groupType.getFieldCount == 1) { // single field, either optional or required new StructType(fields) } else { // multi field repeated group, which we map into an array of structs + if (parquetType.getRepetition == Repetition.REPEATED) { new ArrayType(StructType(fields)) + } else { + new StructType(fields) + } } } } @@ -247,7 +251,11 @@ private[parquet] object ParquetTypesConverter { case _ => None } - def fromDataType(ctype: DataType, name: String, nullable: Boolean = true, inArray: Boolean = false): ParquetType = { + def fromDataType( + ctype: DataType, + name: String, + nullable: Boolean = true, + inArray: Boolean = false): ParquetType = { val repetition = if (inArray) Repetition.REPEATED else { @@ -262,16 +270,17 @@ private[parquet] object ParquetTypesConverter { case ArrayType(elementType: DataType) => { elementType match { case StructType(fields) => { // first case: array of structs - val parquetFieldTypes = fields.map(f => fromDataType(f.dataType, f.name, f.nullable, false)) + val parquetFieldTypes = fields.map( + f => fromDataType(f.dataType, f.name, f.nullable, false)) new ParquetGroupType(repetition, name, ParquetOriginalType.LIST, parquetFieldTypes) //ConversionPatterns.listType(Repetition.REPEATED, name, parquetFieldTypes) } case _ => { // second case: array of primitive types - // TODO: "values" is a generic name but without it the Parquet column path would - // be incomplete and values may be silently dropped; better would be to give - // Array elements a name of some sort (and specify whether they are nullable), - // as in StructField - val parquetElementType = fromDataType(elementType, "values", nullable=false, inArray=true) + val parquetElementType = fromDataType( + elementType, + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + nullable = false, + inArray = true) ConversionPatterns.listType(repetition, name, parquetElementType) } } @@ -281,14 +290,18 @@ private[parquet] object ParquetTypesConverter { val fields = structFields.map { field => fromDataType(field.dataType, field.name, field.nullable) } - new ParquetGroupType(Repetition.REPEATED, name, fields) + new ParquetGroupType(repetition, name, fields) } case _ => sys.error(s"Unsupported datatype $ctype") } } } - def consumeType(consumer: RecordConsumer, ctype: DataType, record: Row, index: Int): Unit = { + def consumeType( + consumer: RecordConsumer, + ctype: DataType, + record: Row, + index: Int): Unit = { ctype match { case StringType => consumer.addBinary( Binary.fromByteArray( @@ -311,11 +324,18 @@ private[parquet] object ParquetTypesConverter { parquetSchema .asGroupType() .getFields - .map(field => new AttributeReference(field.getName, toDataType(field), field.getRepetition != Repetition.REQUIRED)()) + .map( + field => + new AttributeReference( + field.getName, + toDataType(field), + field.getRepetition != Repetition.REQUIRED)()) } def convertFromAttributes(attributes: Seq[Attribute]): MessageType = { - val fields = attributes.map(attribute => fromDataType(attribute.dataType, attribute.name, attribute.nullable)) + val fields = attributes.map( + attribute => + fromDataType(attribute.dataType, attribute.name, attribute.nullable)) new MessageType("root", fields) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 0a527630c7052..981c403ef9cf8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -257,15 +257,21 @@ private[sql] object ParquetTestData { r1.add(0, 1) r1.add(1, 7) val longs = r1.addGroup(2) - longs.add("values", 1.toLong << 32) - longs.add("values", 1.toLong << 33) - longs.add("values", 1.toLong << 34) + longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME , 1.toLong << 32) + longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 1.toLong << 33) + longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 1.toLong << 34) val booleanNumberPairs = r1.addGroup(3) booleanNumberPairs.add("value", 2.5) booleanNumberPairs.add("truth", false) - r1.addGroup(4).addGroup(0).addGroup(0).add("values", 7) - r1.addGroup(4).addGroup(0).addGroup(0).add("values", 8) - r1.addGroup(4).addGroup(0).addGroup(0).add("values", 9) + r1.addGroup(4).addGroup(0).addGroup(0).add( + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + 7) + r1.addGroup(4).addGroup(0).addGroup(0).add( + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + 8) + r1.addGroup(4).addGroup(0).addGroup(0).add( + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + 9) val writeSupport = new TestGroupWriteSupport(schema) val writer = new ParquetWriter[Group](path, writeSupport) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 9f2e6227c778c..77956cd33bace 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -394,7 +394,22 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA ParquetTestData.testNestedFile2, ParquetTestData.testNestedSchema2) val result = getRDD(ParquetTestData.testNestedData2).collect() - assert(result != null) + assert(result.size === 1, "number of top-level rows incorrect") + assert(result(0).size === 5, "number of fields in row incorrect") + assert(result(0)(0) === 1) + assert(result(0)(1) === 7) + assert(result(0)(2).asInstanceOf[ArrayBuffer[Any]].size === 3) + assert(result(0)(2).asInstanceOf[ArrayBuffer[Any]].apply(0) === (1.toLong << 32)) + assert(result(0)(2).asInstanceOf[ArrayBuffer[Any]].apply(1) === (1.toLong << 33)) + assert(result(0)(2).asInstanceOf[ArrayBuffer[Any]].apply(2) === (1.toLong << 34)) + assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].size === 2) + assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].apply(0) === 2.5) + assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].apply(1) === false) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].size === 3) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[Any]].size === 1) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].size === 1) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].apply(0) === 9) + assert(true) } /** From 98219cff6b19c097b59581ca858250eb89a03476 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 2 Apr 2014 17:45:22 +0300 Subject: [PATCH 08/48] added struct converter --- .../spark/sql/parquet/ParquetConverter.scala | 27 ++++++++++++++++--- .../spark/sql/parquet/ParquetTestData.scala | 12 ++++++--- 2 files changed, 32 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 66c9c674fa657..9ac1484686e72 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 @@ -50,12 +50,14 @@ object CatalystConverter { case _ => new CatalystArrayConverter(elementType, fieldIndex, parent) } } - case StructType(fields: Seq[StructField]) => - new CatalystGroupConverter(fields, fieldIndex, parent) - case ctype: NativeType => + case StructType(fields: Seq[StructField]) => { + new CatalystStructConverter(fields, fieldIndex, parent) + } + case ctype: NativeType => { // note: for some reason matching for StringType fails so use this ugly if instead if (ctype == StringType) new CatalystPrimitiveStringConverter(parent, fieldIndex) else new CatalystPrimitiveConverter(parent, fieldIndex) + } case _ => throw new RuntimeException( s"unable to convert datatype ${field.dataType.toString} in CatalystGroupConverter") } @@ -109,7 +111,7 @@ trait CatalystConverter { * @param schema The corresponding Catalyst schema in the form of a list of attributes. */ class CatalystGroupConverter( - private[parquet] val schema: Seq[FieldType], + protected[parquet] val schema: Seq[FieldType], protected[parquet] val index: Int, protected[parquet] val parent: CatalystConverter, protected[parquet] var current: ArrayBuffer[Any], @@ -277,6 +279,23 @@ class CatalystArrayConverter( } } +// this is for multi-element groups of primitive or complex types +// that have repetition level optional or required (so struct fields) +class CatalystStructConverter( + override protected[parquet] val schema: Seq[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) + parent.updateField(index, current) + } +} + // TODO: add MapConverter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 981c403ef9cf8..142c429240a55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -263,13 +263,19 @@ private[sql] object ParquetTestData { val booleanNumberPairs = r1.addGroup(3) booleanNumberPairs.add("value", 2.5) booleanNumberPairs.add("truth", false) - r1.addGroup(4).addGroup(0).addGroup(0).add( + val top_level = r1.addGroup(4) + val second_level_a = top_level.addGroup(0) + val second_level_b = top_level.addGroup(0) + val third_level_aa = second_level_a.addGroup(0) + val third_level_ab = second_level_a.addGroup(0) + val third_level_c = second_level_b.addGroup(0) + third_level_aa.add( CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 7) - r1.addGroup(4).addGroup(0).addGroup(0).add( + third_level_ab.add( CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 8) - r1.addGroup(4).addGroup(0).addGroup(0).add( + third_level_c.add( CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 9) From ee70125324adcff2049f08dfff40273a4ffaffc2 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 3 Apr 2014 18:11:40 +0300 Subject: [PATCH 09/48] fixing one problem with arrayconverter --- .../org/apache/spark/sql/parquet/ParquetConverter.scala | 4 +++- .../org/apache/spark/sql/parquet/ParquetQuerySuite.scala | 9 +++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 9ac1484686e72..85116a22c8532 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 @@ -275,7 +275,9 @@ class CatalystArrayConverter( // TODO: think about reusing the buffer override def end(): Unit = { - if (parent != null) parent.updateField(index, buffer) + assert(parent != null) + parent.updateField(index, buffer) + clearBuffer() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 77956cd33bace..0c40b7adb6db0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -405,11 +405,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].size === 2) assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].apply(0) === 2.5) assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].apply(1) === false) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].size === 3) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].size === 2) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(0).asInstanceOf[ArrayBuffer[Any]].size === 2) assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[Any]].size === 1) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].size === 1) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].apply(0) === 9) - assert(true) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(0).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].apply(0).apply(0) === 7) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(0).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].apply(1).apply(0) === 8) + assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].apply(0).apply(0) === 9) } /** From b7fcc3544efb5c01932cea9a95178132feba74ce Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 4 Apr 2014 19:56:56 +0300 Subject: [PATCH 10/48] Documenting conversions, bugfix, wrappers of Rows --- .../spark/sql/parquet/ParquetConverter.scala | 26 ++++-- .../spark/sql/parquet/ParquetRelation.scala | 84 +++++++++++++++++-- .../spark/sql/parquet/ParquetQuerySuite.scala | 80 ++++++++++++------ 3 files changed, 152 insertions(+), 38 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 85116a22c8532..aa1847851494a 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 @@ -115,7 +115,7 @@ class CatalystGroupConverter( protected[parquet] val index: Int, protected[parquet] val parent: CatalystConverter, protected[parquet] var current: ArrayBuffer[Any], - protected[parquet] var buffer: ArrayBuffer[ArrayBuffer[Any]]) + protected[parquet] var buffer: ArrayBuffer[Row]) extends GroupConverter with CatalystConverter { def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) = @@ -124,7 +124,7 @@ class CatalystGroupConverter( index, parent, current=null, - buffer=new ArrayBuffer[ArrayBuffer[Any]]( + buffer=new ArrayBuffer[Row]( CatalystArrayConverter.INITIAL_ARRAY_SIZE)) // This constructor is used for the root converter only @@ -141,6 +141,7 @@ class CatalystGroupConverter( // Should be only called in root group converter! def getCurrentRecord: Row = { assert(isRootConverter, "getCurrentRecord should only be called in root group converter!") + // TODO: use iterators if possible new GenericRow { override val values: Array[Any] = current.toArray } @@ -155,7 +156,7 @@ class CatalystGroupConverter( override protected[parquet] def clearBuffer(): Unit = { // TODO: reuse buffer? - buffer = new ArrayBuffer[ArrayBuffer[Any]](CatalystArrayConverter.INITIAL_ARRAY_SIZE) + buffer = new ArrayBuffer[Row](CatalystArrayConverter.INITIAL_ARRAY_SIZE) } override def start(): Unit = { @@ -173,8 +174,13 @@ class CatalystGroupConverter( override def end(): Unit = { if (!isRootConverter) { assert(current!=null) // there should be no empty groups - buffer.append(current) - parent.updateField(index, buffer) + buffer.append(new GenericRow { + override val values: Array[Any] = current.toArray + }) + // TODO: use iterators if possible, avoid Row wrapping + parent.updateField(index, new GenericRow { + override val values: Array[Any] = buffer.toArray + }) } } } @@ -276,7 +282,10 @@ class CatalystArrayConverter( // TODO: think about reusing the buffer override def end(): Unit = { assert(parent != null) - parent.updateField(index, buffer) + // TODO: use iterators if possible, avoid Row wrapping + parent.updateField(index, new GenericRow { + override val values: Array[Any] = buffer.toArray + }) clearBuffer() } } @@ -294,7 +303,10 @@ class CatalystStructConverter( // TODO: think about reusing the buffer override def end(): Unit = { assert(!isRootConverter) - parent.updateField(index, current) + // TODO: use iterators if possible, avoid Row wrapping! + parent.updateField(index, new GenericRow { + override val values: Array[Any] = current.toArray + }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index ce5e6d6f8775e..f9d3b268ad662 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -193,6 +193,31 @@ private[parquet] object ParquetTypesConverter { s"Unsupported parquet datatype $parquetType") } + /** + * Converts a given Parquet `Type` into the corresponding + * [[org.apache.spark.sql.catalyst.types.DataType]]. + * + * Note that we apply the following conversion rules: + *
    + *
  • Primitive types are converter to the corresponding primitive type.
  • + *
  • Group types that have a single field with repetition `REPEATED` or themselves + * have repetition level `REPEATED` are converted to an [[ArrayType]] with the + * corresponding field type (possibly primitive) as element type.
  • + *
  • Other group types are converted as follows:
      + *
    • If they have a single field, they are converted into a [[StructType]] with + * the corresponding field type.
    • + *
    • If they have more than one field and repetition level `REPEATED` they are + * converted into an [[ArrayType]] with the corresponding [[StructType]] as complex + * element type.
    • + *
    • Otherwise they are converted into a [[StructType]] with the corresponding + * field types.
  • + *
+ * Note that fields are determined to be `nullable` if and only if their Parquet repetition + * level is not `REQUIRED`. + * + * @param parquetType The type to convert. + * @return The corresponding Catalyst type. + */ def toDataType(parquetType: ParquetType): DataType = { if (parquetType.isPrimitive) { toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) @@ -215,7 +240,9 @@ private[parquet] object ParquetTypesConverter { case _ => { // everything else nested becomes a Struct, unless it has a single repeated field // in which case it becomes an array (this should correspond to the inverse operation of // parquet.schema.ConversionPatterns.listType) - if (groupType.getFieldCount == 1 && groupType.getFields.apply(0).getRepetition == Repetition.REPEATED) { + if (groupType.getFieldCount == 1 && + (groupType.getFields.apply(0).getRepetition == Repetition.REPEATED || + groupType.getRepetition == Repetition.REPEATED)) { val elementType = toDataType(groupType.getFields.apply(0)) new ArrayType(elementType) } else { @@ -225,9 +252,10 @@ private[parquet] object ParquetTypesConverter { ptype.getName, toDataType(ptype), ptype.getRepetition != Repetition.REQUIRED)) - if (groupType.getFieldCount == 1) { // single field, either optional or required + + if (groupType.getFieldCount == 1) { new StructType(fields) - } else { // multi field repeated group, which we map into an array of structs + } else { if (parquetType.getRepetition == Repetition.REPEATED) { new ArrayType(StructType(fields)) } else { @@ -240,6 +268,14 @@ private[parquet] object ParquetTypesConverter { } } + /** + * For a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] return + * the name of the corresponding Parquet primitive type or None if the given type + * is not primitive. + * + * @param ctype The type to convert + * @return The name of the corresponding Parquet primitive type + */ def fromPrimitiveDataType(ctype: DataType): Option[ParquetPrimitiveTypeName] = ctype match { case StringType => Some(ParquetPrimitiveTypeName.BINARY) case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN) @@ -251,6 +287,41 @@ private[parquet] object ParquetTypesConverter { case _ => None } + /** + * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into + * the corrponsing Parquet `Type`. + * + * The conversion follows the rules below: + *
    + *
  • Primitive types are converted into Parquet's primitive types.
  • + *
  • [[org.apache.spark.sql.catalyst.types.StructType]]s are converted + * into Parquet's `GroupType` with the corresponding field types.
  • + *
  • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are handled as follows:
      + *
    • If their element is complex, that is of type + * [[org.apache.spark.sql.catalyst.types.StructType]], they are converted + * into a `GroupType` with the corresponding field types of the struct and + * original type of the `GroupType` is set to `LIST`.
    • + *
    • Otherwise, that is they contain a primitive they are converted into a `GroupType` + * that is also a list but has only a single field of the type corresponding to + * the element type.
  • + *
+ * Parquet's repetition level is set according to the following rule: + *
    + *
  • If the call to `fromDataType` is recursive inside an enclosing `ArrayType`, then + * the repetition level is set to `REPEATED`.
  • + *
  • Otherwise, if the attribute whose type is converted is `nullable`, the Parquet + * type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.
  • + *
+ * The single expection to this rule is an [[org.apache.spark.sql.catalyst.types.ArrayType]] + * that contains a [[org.apache.spark.sql.catalyst.types.StructType]], whose repetition level + * is always set to `REPEATED`. + * + @param ctype The type to convert. + * @param name The name of the [[org.apache.spark.sql.catalyst.expressions.Attribute]] whose type is converted + * @param nullable When true indicates that the attribute is nullable + * @param inArray When true indicates that this is a nested attribute inside an array. + * @return The corresponding Parquet type. + */ def fromDataType( ctype: DataType, name: String, @@ -271,8 +342,9 @@ private[parquet] object ParquetTypesConverter { elementType match { case StructType(fields) => { // first case: array of structs val parquetFieldTypes = fields.map( - f => fromDataType(f.dataType, f.name, f.nullable, false)) - new ParquetGroupType(repetition, name, ParquetOriginalType.LIST, parquetFieldTypes) + f => fromDataType(f.dataType, f.name, f.nullable, inArray = false)) + assert(fields.size > 1, "Found struct inside array with a single field.. error parsin Catalyst schema") + new ParquetGroupType(Repetition.REPEATED, name, ParquetOriginalType.LIST, parquetFieldTypes) //ConversionPatterns.listType(Repetition.REPEATED, name, parquetFieldTypes) } case _ => { // second case: array of primitive types @@ -288,7 +360,7 @@ private[parquet] object ParquetTypesConverter { // TODO: test structs inside arrays case StructType(structFields) => { val fields = structFields.map { - field => fromDataType(field.dataType, field.name, field.nullable) + field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) } new ParquetGroupType(repetition, name, fields) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 0c40b7adb6db0..ac1d595ecb44d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -54,6 +54,10 @@ case class OptionalReflectData( doubleField: Option[Double], booleanField: Option[Boolean]) +case class Nested(i: Int, s: String) + +case class Data(array: Seq[Int], nested: Nested) + class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { import TestData._ TestData // Load test data tables. @@ -366,6 +370,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Importing nested Parquet file (Addressbook)") { + implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] ParquetTestData.readNestedFile( ParquetTestData.testNestedFile1, ParquetTestData.testNestedSchema1) @@ -374,22 +379,23 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result.size === 2) val first_record = result(0) val second_record = result(1) - val first_owner_numbers = result(0).apply(1).asInstanceOf[ArrayBuffer[Any]] - val first_contacts = result(0).apply(2).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]] + val first_owner_numbers = result(0)(1) + val first_contacts = result(0)(2) assert(first_record.size === 3) - assert(second_record.apply(1) === null) - assert(second_record.apply(2) === null) - assert(second_record.apply(0) === "A. Nonymous") - assert(first_record.apply(0) === "Julien Le Dem") - assert(first_owner_numbers.apply(0) === "555 123 4567") - assert(first_owner_numbers.apply(2) === "XXX XXX XXXX") - assert(first_contacts.apply(0).size === 2) - assert(first_contacts.apply(0).apply(0) === "Dmitriy Ryaboy") - assert(first_contacts.apply(0).apply(1) === "555 987 6543") - assert(first_contacts.apply(1).apply(0) === "Chris Aniszczyk") + assert(second_record(1) === null) + assert(second_record(2) === null) + assert(second_record(0) === "A. Nonymous") + assert(first_record(0) === "Julien Le Dem") + assert(first_owner_numbers(0) === "555 123 4567") + assert(first_owner_numbers(2) === "XXX XXX XXXX") + assert(first_contacts(0).size === 2) + assert(first_contacts(0)(0) === "Dmitriy Ryaboy") + assert(first_contacts(0)(1) === "555 987 6543") + assert(first_contacts(1)(0) === "Chris Aniszczyk") } test("Importing nested Parquet file (nested numbers)") { + implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] ParquetTestData.readNestedFile( ParquetTestData.testNestedFile2, ParquetTestData.testNestedSchema2) @@ -398,19 +404,43 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(0).size === 5, "number of fields in row incorrect") assert(result(0)(0) === 1) assert(result(0)(1) === 7) - assert(result(0)(2).asInstanceOf[ArrayBuffer[Any]].size === 3) - assert(result(0)(2).asInstanceOf[ArrayBuffer[Any]].apply(0) === (1.toLong << 32)) - assert(result(0)(2).asInstanceOf[ArrayBuffer[Any]].apply(1) === (1.toLong << 33)) - assert(result(0)(2).asInstanceOf[ArrayBuffer[Any]].apply(2) === (1.toLong << 34)) - assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].size === 2) - assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].apply(0) === 2.5) - assert(result(0)(3).asInstanceOf[ArrayBuffer[Any]].apply(1) === false) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].size === 2) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(0).asInstanceOf[ArrayBuffer[Any]].size === 2) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[Any]].size === 1) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(0).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].apply(0).apply(0) === 7) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(0).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].apply(1).apply(0) === 8) - assert(result(0)(4).asInstanceOf[ArrayBuffer[Any]].apply(1).asInstanceOf[ArrayBuffer[ArrayBuffer[Any]]].apply(0).apply(0) === 9) + assert(result(0)(2).size === 3) + assert(result(0)(2)(0) === (1.toLong << 32)) + assert(result(0)(2)(1) === (1.toLong << 33)) + assert(result(0)(2)(2) === (1.toLong << 34)) + assert(result(0)(3).size === 2) + assert(result(0)(3)(0) === 2.5) + assert(result(0)(3)(1) === false) + assert(result(0)(4).size === 2) + assert(result(0)(4)(0).size === 2) + assert(result(0)(4)(1).size === 1) + assert(result(0)(4)(0)(0)(0) === 7) + assert(result(0)(4)(0)(1)(0) === 8) + assert(result(0)(4)(1)(0)(0) === 9) + } + + test("Simple query on addressbook") { + val data = TestSQLContext.parquetFile(ParquetTestData.testNestedFile1.toString).toSchemaRDD + val tmp = data.where('owner === "Julien Le Dem").select('owner as 'a, 'contacts as 'c).collect() + assert(tmp.size === 1) + assert(tmp(0)(0) === "Julien Le Dem") + } + + test("Simple query on nested int data") { + implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] + val data = TestSQLContext.parquetFile(ParquetTestData.testNestedFile2.toString).toSchemaRDD + data.registerAsTable("data") + val tmp = sql("SELECT booleanNumberPairs.value, booleanNumberPairs.truth FROM data").collect() + assert(tmp(0)(0) === 2.5) + assert(tmp(0)(1) === false) + val result = sql("SELECT outerouter FROM data").collect() + // TODO: why does this not work? + //val result = sql("SELECT outerouter.values FROM data").collect() + // TODO: .. or this: + // val result = sql("SELECT outerouter[0] FROM data").collect() + assert(result(0)(0)(0)(0)(0) === 7) + assert(result(0)(0)(0)(1)(0) === 8) + assert(result(0)(0)(1)(0)(0) === 9) } /** From 6dbc9b73d03566bed365cd1416ca240799df7747 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 6 Apr 2014 17:19:23 +0300 Subject: [PATCH 11/48] Fixing some problems intruduced during rebase --- .../spark/sql/parquet/ParquetConverter.scala | 31 +-- .../spark/sql/parquet/ParquetRelation.scala | 2 +- .../sql/parquet/ParquetTableSupport.scala | 223 ------------------ .../spark/sql/parquet/ParquetQuerySuite.scala | 44 +++- 4 files changed, 49 insertions(+), 251 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index aa1847851494a..8ec0f168f1fba 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} import org.apache.spark.sql.parquet.CatalystConverter.FieldType -object CatalystConverter { +private[parquet] object CatalystConverter { // The type internally used for fields type FieldType = StructField @@ -55,11 +55,14 @@ object CatalystConverter { } case ctype: NativeType => { // note: for some reason matching for StringType fails so use this ugly if instead - if (ctype == StringType) new CatalystPrimitiveStringConverter(parent, fieldIndex) - else new CatalystPrimitiveConverter(parent, fieldIndex) + if (ctype == StringType) { + new CatalystPrimitiveStringConverter(parent, fieldIndex) + } else { + new CatalystPrimitiveConverter(parent, fieldIndex) + } } case _ => throw new RuntimeException( - s"unable to convert datatype ${field.dataType.toString} in CatalystGroupConverter") + s"unable to convert datatype ${field.dataType.toString} in CatalystConverter") } } } @@ -142,9 +145,7 @@ class CatalystGroupConverter( def getCurrentRecord: Row = { assert(isRootConverter, "getCurrentRecord should only be called in root group converter!") // TODO: use iterators if possible - new GenericRow { - override val values: Array[Any] = current.toArray - } + new GenericRow(current.toArray) } override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) @@ -174,13 +175,9 @@ class CatalystGroupConverter( override def end(): Unit = { if (!isRootConverter) { assert(current!=null) // there should be no empty groups - buffer.append(new GenericRow { - override val values: Array[Any] = current.toArray - }) + buffer.append(new GenericRow(current.toArray)) // TODO: use iterators if possible, avoid Row wrapping - parent.updateField(index, new GenericRow { - override val values: Array[Any] = buffer.toArray - }) + parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]])) } } } @@ -283,9 +280,7 @@ class CatalystArrayConverter( override def end(): Unit = { assert(parent != null) // TODO: use iterators if possible, avoid Row wrapping - parent.updateField(index, new GenericRow { - override val values: Array[Any] = buffer.toArray - }) + parent.updateField(index, new GenericRow(buffer.toArray)) clearBuffer() } } @@ -304,9 +299,7 @@ class CatalystStructConverter( override def end(): Unit = { assert(!isRootConverter) // TODO: use iterators if possible, avoid Row wrapping! - parent.updateField(index, new GenericRow { - override val values: Array[Any] = current.toArray - }) + parent.updateField(index, new GenericRow(current.toArray)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index f9d3b268ad662..0777c428cc370 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.parquet -import java.io.IOException, +import java.io.IOException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} 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 cccdfd99aacc5..ba2dbe12c02b7 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 @@ -148,226 +148,3 @@ private[parquet] object RowWriteSupport { val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema" } -private[parquet] object CatalystConverter { - type FieldType = StructField - - protected[parquet] def createConverter(field: FieldType, fieldIndex: Int, parent: CatalystConverter): Converter = { - val fieldType: DataType = field.dataType - fieldType match { - case ArrayType(elementType: DataType) => { - elementType match { - case StructType(fields) => - if (fields.size > 1) new CatalystGroupConverter(fields, fieldIndex, parent) - else new CatalystArrayConverter(fields(0).dataType, fieldIndex, parent) - case _ => new CatalystArrayConverter(elementType, fieldIndex, parent) - } - } - case StructType(fields: Seq[StructField]) => - new CatalystGroupConverter(fields, fieldIndex, parent) - case ctype: NativeType => - // note: for some reason matching for StringType fails so use this ugly if instead - if (ctype == StringType) { - new CatalystPrimitiveStringConverter(parent, fieldIndex) - } else { - new CatalystPrimitiveConverter(parent, fieldIndex) - } - case _ => throw new RuntimeException( - s"unable to convert datatype ${field.dataType.toString} in CatalystGroupConverter") - } - } -} - -trait CatalystConverter { - - // 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 - - // for child converters to update upstream values - protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit - - // TODO: in the future consider using specific methods to avoid autoboxing - 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 updateLong(fieldIndex: Int, value: Long): 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: Binary): Unit = - updateField(fieldIndex, value.toStringUsingUTF8) - - protected[parquet] def isRootConverter: Boolean = parent == null - - protected[parquet] def clearBuffer(): Unit -} - -/** - * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record - * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. - * - * @param schema The corresponding Catalyst schema in the form of a list of attributes. - */ -class CatalystGroupConverter( - private[parquet] val schema: Seq[FieldType], - protected[parquet] val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var current: ArrayBuffer[Any], - protected[parquet] var buffer: ArrayBuffer[ArrayBuffer[Any]]) extends GroupConverter with CatalystConverter { - - def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) = - this(schema, index, parent, current=null, buffer=new ArrayBuffer[ArrayBuffer[Any]](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) - - // This constructor is used for the root converter only - def this(attributes: Seq[Attribute]) = - this(attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), 0, null) - - protected [parquet] val converters: Array[Converter] = - schema.map(field => CatalystConverter.createConverter(field, schema.indexOf(field), this)).toArray - - override val size = schema.size - - // Should be only called in root group converter! - def getCurrentRecord: Row = { - assert(isRootConverter, "getCurrentRecord should only be called in root group converter!") - new GenericRow { - override val values: Array[Any] = 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 = { - // TODO: reuse buffer? - buffer = new ArrayBuffer[ArrayBuffer[Any]](CatalystArrayConverter.INITIAL_ARRAY_SIZE) - } - - override def start(): Unit = { - // TODO: reuse buffer? - // Allocate new array in the root converter (others will be called clearBuffer() on) - current = ArrayBuffer.fill(schema.length)(null) - converters.foreach { - converter => if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer - } - } - } - - // TODO: think about reusing the buffer - override def end(): Unit = { - if (!isRootConverter) { - assert(current!=null) // there should be no empty groups - buffer.append(current) - parent.updateField(index, buffer) - } - } -} - -/** - * 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 { - // TODO: consider refactoring these together with ParquetTypesConverter - 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 strings (fixed-length byte arrays) - * into Catalyst Strings. - * - * @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) { - override def addBinary(value: Binary): Unit = - parent.updateString(fieldIndex, value) -} - -object CatalystArrayConverter { - val INITIAL_ARRAY_SIZE = 20 -} - -// this is for single-element groups of primitive or complex types -// Note: AvroParquet only uses arrays for primitive types (?) -class CatalystArrayConverter( - val elementType: DataType, - val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var buffer: Buffer[Any]) - extends GroupConverter with CatalystConverter { - // TODO: In the future consider using native arrays instead of buffer for primitive types for - // performance reasons (autoboxing) - - 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("values", elementType, false), fieldIndex=0, parent=this) - - override def getConverter(fieldIndex: Int): Converter = converter - - override val size = 1 // arrays have only one (repeated) field, which is its elements - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = buffer += value - - override protected[parquet] def clearBuffer(): Unit = { - // TODO: reuse buffer? - buffer = new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE) - } - - override def start(): Unit = { - if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer - } - } - - // TODO: think about reusing the buffer - override def end(): Unit = { - if (parent != null) parent.updateField(index, buffer) - } -} - -// TODO: add MapConverter diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index ac1d595ecb44d..e21d051ba30ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -205,8 +205,26 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(file) } - test("insert (appending) to same table via Scala API") { - sql("INSERT INTO testsource SELECT * FROM testsource") + test("Insert (overwrite) via Scala API") { + val dirname = Utils.createTempDir() + val source_rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + source_rdd.registerAsTable("source") + val dest_rdd = createParquetFile(dirname.toString, ("key", IntegerType), ("value", StringType)) + dest_rdd.registerAsTable("dest") + sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() + val rdd_copy1 = sql("SELECT * FROM dest").collect() + assert(rdd_copy1.size === 100) + assert(rdd_copy1(0).apply(0) === 1) + assert(rdd_copy1(0).apply(1) === "val_1") + sql("INSERT INTO dest SELECT * FROM source").collect() + val rdd_copy2 = sql("SELECT * FROM dest").collect() + assert(rdd_copy2.size === 200) + Utils.deleteRecursively(dirname) + } + + test("Insert (appending) to same table via Scala API") { + sql("INSERT INTO testsource SELECT * FROM testsource").collect() val double_rdd = sql("SELECT * FROM testsource").collect() assert(double_rdd != null) assert(double_rdd.size === 30) @@ -372,9 +390,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Importing nested Parquet file (Addressbook)") { implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] ParquetTestData.readNestedFile( - ParquetTestData.testNestedFile1, + ParquetTestData.testNestedDir1, ParquetTestData.testNestedSchema1) - val result = getRDD(ParquetTestData.testNestedData1).collect() + val result = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir1.toString) + .toSchemaRDD + .collect() assert(result != null) assert(result.size === 2) val first_record = result(0) @@ -397,9 +418,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Importing nested Parquet file (nested numbers)") { implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] ParquetTestData.readNestedFile( - ParquetTestData.testNestedFile2, + ParquetTestData.testNestedDir2, ParquetTestData.testNestedSchema2) - val result = getRDD(ParquetTestData.testNestedData2).collect() + val result = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir2.toString) + .toSchemaRDD + .collect() assert(result.size === 1, "number of top-level rows incorrect") assert(result(0).size === 5, "number of fields in row incorrect") assert(result(0)(0) === 1) @@ -420,7 +444,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Simple query on addressbook") { - val data = TestSQLContext.parquetFile(ParquetTestData.testNestedFile1.toString).toSchemaRDD + val data = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir1.toString) + .toSchemaRDD val tmp = data.where('owner === "Julien Le Dem").select('owner as 'a, 'contacts as 'c).collect() assert(tmp.size === 1) assert(tmp(0)(0) === "Julien Le Dem") @@ -428,7 +454,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Simple query on nested int data") { implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] - val data = TestSQLContext.parquetFile(ParquetTestData.testNestedFile2.toString).toSchemaRDD + val data = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir2.toString) + .toSchemaRDD data.registerAsTable("data") val tmp = sql("SELECT booleanNumberPairs.value, booleanNumberPairs.truth FROM data").collect() assert(tmp(0)(0) === 2.5) From f8f8911906c481b203ffc33165f604237568c9c4 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 6 Apr 2014 19:04:44 +0300 Subject: [PATCH 12/48] For primitive rows fall back to more efficient converter, code reorg --- .../spark/sql/parquet/ParquetConverter.scala | 107 +++++- .../spark/sql/parquet/ParquetRelation.scala | 341 +---------------- .../sql/parquet/ParquetTableSupport.scala | 14 +- .../spark/sql/parquet/ParquetTypes.scala | 355 ++++++++++++++++++ 4 files changed, 464 insertions(+), 353 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala 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 8ec0f168f1fba..2eee9a4c30cf7 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 @@ -20,7 +20,9 @@ package org.apache.spark.sql.parquet import scala.collection.mutable.{Buffer, ArrayBuffer} import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} +import parquet.schema.MessageType +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} import org.apache.spark.sql.parquet.CatalystConverter.FieldType @@ -65,9 +67,19 @@ private[parquet] object CatalystConverter { s"unable to convert datatype ${field.dataType.toString} in CatalystConverter") } } + + protected[parquet] def createRootConverter(parquetSchema: MessageType): CatalystConverter = { + val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) + // For non-nested types we use the optimized Row converter + if (attributes.forall(a => ParquetTypesConverter.isPrimitiveType(a.dataType))) { + new MutableRowGroupConverter(attributes) + } else { + new CatalystGroupConverter(attributes) + } + } } -trait CatalystConverter { +private[parquet] trait CatalystConverter { // the number of fields this group has protected[parquet] val size: Int @@ -80,7 +92,6 @@ trait CatalystConverter { // for child converters to update upstream values protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit - // TODO: in the future consider using specific methods to avoid autoboxing protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = updateField(fieldIndex, value) @@ -105,6 +116,9 @@ trait CatalystConverter { protected[parquet] def isRootConverter: Boolean = parent == null protected[parquet] def clearBuffer(): Unit + + // Should be only called in root group converter! + def getCurrentRecord: Row } /** @@ -113,7 +127,7 @@ trait CatalystConverter { * * @param schema The corresponding Catalyst schema in the form of a list of attributes. */ -class CatalystGroupConverter( +private[parquet] class CatalystGroupConverter( protected[parquet] val schema: Seq[FieldType], protected[parquet] val index: Int, protected[parquet] val parent: CatalystConverter, @@ -182,13 +196,86 @@ class CatalystGroupConverter( } } +/** + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his + * converter is optimized for rows of primitive types (non-nested records). + */ +private[parquet] class MutableRowGroupConverter( + protected[parquet] val schema: Seq[FieldType], + protected[parquet] var current: ParquetRelation.RowType) + extends GroupConverter with CatalystConverter { + + // This constructor is used for the root converter only + def this(attributes: Seq[Attribute]) = + this( + attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), + new ParquetRelation.RowType(attributes.length)) + + protected [parquet] val converters: Array[Converter] = + schema.map(field => + CatalystConverter.createConverter(field, schema.indexOf(field), this)) + .toArray + + override val size = schema.size + + override val index = 0 + + override val parent = null + + // Should be only called in root group converter! + def getCurrentRecord: ParquetRelation.RowType = 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 < schema.length) { + current.setNullAt(i) + i = i + 1 + } + } + + override def end(): Unit = {} + + // Overriden 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 updateLong(fieldIndex: Int, value: Long): Unit = + current.setLong(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: Binary): Unit = + current.setString(fieldIndex, value.toStringUsingUTF8) +} + /** * 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. */ -class CatalystPrimitiveConverter( +private[parquet] class CatalystPrimitiveConverter( parent: CatalystConverter, fieldIndex: Int) extends PrimitiveConverter { // TODO: consider refactoring these together with ParquetTypesConverter @@ -218,7 +305,7 @@ class CatalystPrimitiveConverter( * @param parent The parent group converter. * @param fieldIndex The index inside the record. */ -class CatalystPrimitiveStringConverter( +private[parquet] class CatalystPrimitiveStringConverter( parent: CatalystConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { @@ -232,7 +319,7 @@ object CatalystArrayConverter { // this is for single-element groups of primitive or complex types // Note: AvroParquet only uses arrays for primitive types (?) -class CatalystArrayConverter( +private[parquet] class CatalystArrayConverter( val elementType: DataType, val index: Int, protected[parquet] val parent: CatalystConverter, @@ -283,11 +370,14 @@ class CatalystArrayConverter( parent.updateField(index, new GenericRow(buffer.toArray)) clearBuffer() } + + // Should be only called in root group converter! + override def getCurrentRecord: Row = throw new UnsupportedOperationException } // this is for multi-element groups of primitive or complex types // that have repetition level optional or required (so struct fields) -class CatalystStructConverter( +private[parquet] class CatalystStructConverter( override protected[parquet] val schema: Seq[FieldType], override protected[parquet] val index: Int, override protected[parquet] val parent: CatalystConverter) @@ -301,6 +391,9 @@ class CatalystStructConverter( // TODO: use iterators if possible, avoid Row wrapping! parent.updateField(index, new GenericRow(current.toArray)) } + + // Should be only called in root group converter! + override def getCurrentRecord: Row = throw new UnsupportedOperationException } // TODO: add MapConverter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 0777c428cc370..84ccd8ccc9661 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -20,26 +20,16 @@ package org.apache.spark.sql.parquet import java.io.IOException import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.permission.FsAction -import org.apache.hadoop.mapreduce.Job -import parquet.hadoop.util.ContextUtil -import parquet.hadoop.{ParquetOutputFormat, Footer, ParquetFileWriter, ParquetFileReader} - -import parquet.hadoop.metadata.{CompressionCodecName, FileMetaData, ParquetMetadata} -import parquet.io.api.{Binary, RecordConsumer} -import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser, GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns} -import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} -import parquet.schema.Type.Repetition +import parquet.hadoop.ParquetOutputFormat +import parquet.hadoop.metadata.CompressionCodecName +import parquet.schema.MessageType import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} -import org.apache.spark.sql.catalyst.types._ - -// Implicits -import scala.collection.JavaConversions._ /** * Relation that consists of data stored in a Parquet columnar format. @@ -171,324 +161,3 @@ private[sql] object ParquetRelation { path } } - -private[parquet] object ParquetTypesConverter { - def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass - - def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { - // for now map binary to string type - // TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema - case ParquetPrimitiveTypeName.BINARY => StringType - case ParquetPrimitiveTypeName.BOOLEAN => BooleanType - case ParquetPrimitiveTypeName.DOUBLE => DoubleType - case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType) - case ParquetPrimitiveTypeName.FLOAT => FloatType - case ParquetPrimitiveTypeName.INT32 => IntegerType - case ParquetPrimitiveTypeName.INT64 => LongType - case ParquetPrimitiveTypeName.INT96 => - // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? - sys.error("Warning: potential loss of precision: converting INT96 to long") - LongType - case _ => sys.error( - s"Unsupported parquet datatype $parquetType") - } - - /** - * Converts a given Parquet `Type` into the corresponding - * [[org.apache.spark.sql.catalyst.types.DataType]]. - * - * Note that we apply the following conversion rules: - *
    - *
  • Primitive types are converter to the corresponding primitive type.
  • - *
  • Group types that have a single field with repetition `REPEATED` or themselves - * have repetition level `REPEATED` are converted to an [[ArrayType]] with the - * corresponding field type (possibly primitive) as element type.
  • - *
  • Other group types are converted as follows:
      - *
    • If they have a single field, they are converted into a [[StructType]] with - * the corresponding field type.
    • - *
    • If they have more than one field and repetition level `REPEATED` they are - * converted into an [[ArrayType]] with the corresponding [[StructType]] as complex - * element type.
    • - *
    • Otherwise they are converted into a [[StructType]] with the corresponding - * field types.
  • - *
- * Note that fields are determined to be `nullable` if and only if their Parquet repetition - * level is not `REQUIRED`. - * - * @param parquetType The type to convert. - * @return The corresponding Catalyst type. - */ - def toDataType(parquetType: ParquetType): DataType = { - if (parquetType.isPrimitive) { - toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) - } - else { - val groupType = parquetType.asGroupType() - parquetType.getOriginalType match { - // if the schema was constructed programmatically there may be hints how to convert - // it inside the metadata via the OriginalType field - case ParquetOriginalType.LIST => { // TODO: check enums! - val fields = groupType.getFields.map { - field => new StructField(field.getName, toDataType(field), field.getRepetition != Repetition.REQUIRED) - } - if (fields.size == 1) { - new ArrayType(fields.apply(0).dataType) - } else { - new ArrayType(StructType(fields)) - } - } - case _ => { // everything else nested becomes a Struct, unless it has a single repeated field - // in which case it becomes an array (this should correspond to the inverse operation of - // parquet.schema.ConversionPatterns.listType) - if (groupType.getFieldCount == 1 && - (groupType.getFields.apply(0).getRepetition == Repetition.REPEATED || - groupType.getRepetition == Repetition.REPEATED)) { - val elementType = toDataType(groupType.getFields.apply(0)) - new ArrayType(elementType) - } else { - val fields = groupType - .getFields - .map(ptype => new StructField( - ptype.getName, - toDataType(ptype), - ptype.getRepetition != Repetition.REQUIRED)) - - if (groupType.getFieldCount == 1) { - new StructType(fields) - } else { - if (parquetType.getRepetition == Repetition.REPEATED) { - new ArrayType(StructType(fields)) - } else { - new StructType(fields) - } - } - } - } - } - } - } - - /** - * For a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] return - * the name of the corresponding Parquet primitive type or None if the given type - * is not primitive. - * - * @param ctype The type to convert - * @return The name of the corresponding Parquet primitive type - */ - def fromPrimitiveDataType(ctype: DataType): Option[ParquetPrimitiveTypeName] = ctype match { - case StringType => Some(ParquetPrimitiveTypeName.BINARY) - case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN) - case DoubleType => Some(ParquetPrimitiveTypeName.DOUBLE) - case ArrayType(ByteType) => Some(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) - case FloatType => Some(ParquetPrimitiveTypeName.FLOAT) - case IntegerType => Some(ParquetPrimitiveTypeName.INT32) - case LongType => Some(ParquetPrimitiveTypeName.INT64) - case _ => None - } - - /** - * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into - * the corrponsing Parquet `Type`. - * - * The conversion follows the rules below: - *
    - *
  • Primitive types are converted into Parquet's primitive types.
  • - *
  • [[org.apache.spark.sql.catalyst.types.StructType]]s are converted - * into Parquet's `GroupType` with the corresponding field types.
  • - *
  • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are handled as follows:
      - *
    • If their element is complex, that is of type - * [[org.apache.spark.sql.catalyst.types.StructType]], they are converted - * into a `GroupType` with the corresponding field types of the struct and - * original type of the `GroupType` is set to `LIST`.
    • - *
    • Otherwise, that is they contain a primitive they are converted into a `GroupType` - * that is also a list but has only a single field of the type corresponding to - * the element type.
  • - *
- * Parquet's repetition level is set according to the following rule: - *
    - *
  • If the call to `fromDataType` is recursive inside an enclosing `ArrayType`, then - * the repetition level is set to `REPEATED`.
  • - *
  • Otherwise, if the attribute whose type is converted is `nullable`, the Parquet - * type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.
  • - *
- * The single expection to this rule is an [[org.apache.spark.sql.catalyst.types.ArrayType]] - * that contains a [[org.apache.spark.sql.catalyst.types.StructType]], whose repetition level - * is always set to `REPEATED`. - * - @param ctype The type to convert. - * @param name The name of the [[org.apache.spark.sql.catalyst.expressions.Attribute]] whose type is converted - * @param nullable When true indicates that the attribute is nullable - * @param inArray When true indicates that this is a nested attribute inside an array. - * @return The corresponding Parquet type. - */ - def fromDataType( - ctype: DataType, - name: String, - nullable: Boolean = true, - inArray: Boolean = false): ParquetType = { - val repetition = - if (inArray) Repetition.REPEATED - else { - if (nullable) Repetition.OPTIONAL - else Repetition.REQUIRED - } - val primitiveType = fromPrimitiveDataType(ctype) - if (primitiveType.isDefined) { - new ParquetPrimitiveType(repetition, primitiveType.get, name) - } else { - ctype match { - case ArrayType(elementType: DataType) => { - elementType match { - case StructType(fields) => { // first case: array of structs - val parquetFieldTypes = fields.map( - f => fromDataType(f.dataType, f.name, f.nullable, inArray = false)) - assert(fields.size > 1, "Found struct inside array with a single field.. error parsin Catalyst schema") - new ParquetGroupType(Repetition.REPEATED, name, ParquetOriginalType.LIST, parquetFieldTypes) - //ConversionPatterns.listType(Repetition.REPEATED, name, parquetFieldTypes) - } - case _ => { // second case: array of primitive types - val parquetElementType = fromDataType( - elementType, - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - nullable = false, - inArray = true) - ConversionPatterns.listType(repetition, name, parquetElementType) - } - } - } - // TODO: test structs inside arrays - case StructType(structFields) => { - val fields = structFields.map { - field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) - } - new ParquetGroupType(repetition, name, fields) - } - case _ => sys.error(s"Unsupported datatype $ctype") - } - } - } - - def consumeType( - consumer: RecordConsumer, - ctype: DataType, - record: Row, - index: Int): Unit = { - ctype match { - case StringType => consumer.addBinary( - Binary.fromByteArray( - record(index).asInstanceOf[String].getBytes("utf-8") - ) - ) - case IntegerType => consumer.addInteger(record.getInt(index)) - case LongType => consumer.addLong(record.getLong(index)) - case DoubleType => consumer.addDouble(record.getDouble(index)) - case FloatType => consumer.addFloat(record.getFloat(index)) - case BooleanType => consumer.addBoolean(record.getBoolean(index)) - case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer") - } - } - - def getSchema(schemaString: String) : MessageType = - MessageTypeParser.parseMessageType(schemaString) - - def convertToAttributes(parquetSchema: ParquetType): Seq[Attribute] = { - parquetSchema - .asGroupType() - .getFields - .map( - field => - new AttributeReference( - field.getName, - toDataType(field), - field.getRepetition != Repetition.REQUIRED)()) - } - - def convertFromAttributes(attributes: Seq[Attribute]): MessageType = { - val fields = attributes.map( - attribute => - fromDataType(attribute.dataType, attribute.name, attribute.nullable)) - new MessageType("root", fields) - } - - def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { - if (origPath == null) { - throw new IllegalArgumentException("Unable to write Parquet metadata: path is null") - } - val fs = origPath.getFileSystem(conf) - if (fs == null) { - throw new IllegalArgumentException( - s"Unable to write Parquet metadata: path $origPath is incorrectly formatted") - } - val path = origPath.makeQualified(fs) - if (fs.exists(path) && !fs.getFileStatus(path).isDir) { - throw new IllegalArgumentException(s"Expected to write to directory $path but found file") - } - val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - if (fs.exists(metadataPath)) { - try { - fs.delete(metadataPath, true) - } catch { - case e: IOException => - throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE at $metadataPath") - } - } - val extraMetadata = new java.util.HashMap[String, String]() - extraMetadata.put("path", path.toString) - // TODO: add extra data, e.g., table name, date, etc.? - - val parquetSchema: MessageType = - ParquetTypesConverter.convertFromAttributes(attributes) - val metaData: FileMetaData = new FileMetaData( - parquetSchema, - extraMetadata, - "Spark") - - ParquetRelation.enableLogForwarding() - ParquetFileWriter.writeMetadataFile( - conf, - path, - new Footer(path, new ParquetMetadata(metaData, Nil)) :: Nil) - } - - /** - * Try to read Parquet metadata at the given Path. We first see if there is a summary file - * in the parent directory. If so, this is used. Else we read the actual footer at the given - * location. - * @param origPath The path at which we expect one (or more) Parquet files. - * @return The `ParquetMetadata` containing among other things the schema. - */ - def readMetaData(origPath: Path): ParquetMetadata = { - if (origPath == null) { - throw new IllegalArgumentException("Unable to read Parquet metadata: path is null") - } - val job = new Job() - // TODO: since this is called from ParquetRelation (LogicalPlan) we don't have access - // to SparkContext's hadoopConfig; in principle the default FileSystem may be different(?!) - val conf = ContextUtil.getConfiguration(job) - val fs: FileSystem = origPath.getFileSystem(conf) - if (fs == null) { - throw new IllegalArgumentException(s"Incorrectly formatted Parquet metadata path $origPath") - } - val path = origPath.makeQualified(fs) - if (!fs.getFileStatus(path).isDir) { - throw new IllegalArgumentException( - s"Expected $path for be a directory with Parquet files/metadata") - } - ParquetRelation.enableLogForwarding() - val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - // if this is a new table that was just created we will find only the metadata file - if (fs.exists(metadataPath) && fs.isFile(metadataPath)) { - ParquetFileReader.readFooter(conf, metadataPath) - } else { - // there may be one or more Parquet files in the given directory - val footers = ParquetFileReader.readFooters(conf, fs.getFileStatus(path)) - // TODO: for now we assume that all footers (if there is more than one) have identical - // metadata; we may want to add a check here at some point - if (footers.size() == 0) { - throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path") - } - footers(0).getParquetMetadata - } - } -} 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 ba2dbe12c02b7..ede8fb6204664 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,8 +17,6 @@ package org.apache.spark.sql.parquet -import collection.mutable.{ArrayBuffer, Buffer} - import org.apache.hadoop.conf.Configuration import parquet.column.ParquetProperties @@ -29,26 +27,22 @@ import parquet.io.api._ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, Row} -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.parquet.CatalystConverter.FieldType -import org.apache.spark.sql.parquet.ParquetRelation.RowType -import scala.collection.mutable +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} /** * A `parquet.io.api.RecordMaterializer` for Rows. * *@param root The root group converter for the record. */ -private[parquet] class RowRecordMaterializer(root: CatalystGroupConverter) +private[parquet] class RowRecordMaterializer(root: CatalystConverter) extends RecordMaterializer[Row] { def this(parquetSchema: MessageType) = - this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema))) + this(CatalystConverter.createRootConverter(parquetSchema)) override def getCurrentRecord: Row = root.getCurrentRecord - override def getRootConverter: GroupConverter = root + override def getRootConverter: GroupConverter = root.asInstanceOf[GroupConverter] } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala new file mode 100644 index 0000000000000..04e4feffcc89b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -0,0 +1,355 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.parquet + +import java.io.IOException + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.mapreduce.Job + +import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter} +import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData} +import parquet.hadoop.util.ContextUtil +import parquet.io.api.{Binary, RecordConsumer} +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser, GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns} +import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} +import parquet.schema.Type.Repetition + +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute, Row} +import org.apache.spark.sql.catalyst.types._ + +// Implicits +import scala.collection.JavaConversions._ + +private[parquet] object ParquetTypesConverter { + def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass + + def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { + case ParquetPrimitiveTypeName.BINARY => StringType + case ParquetPrimitiveTypeName.BOOLEAN => BooleanType + case ParquetPrimitiveTypeName.DOUBLE => DoubleType + case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType) + case ParquetPrimitiveTypeName.FLOAT => FloatType + case ParquetPrimitiveTypeName.INT32 => IntegerType + case ParquetPrimitiveTypeName.INT64 => LongType + case ParquetPrimitiveTypeName.INT96 => + // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? + sys.error("Potential loss of precision: cannot convert INT96") + case _ => sys.error( + s"Unsupported parquet datatype $parquetType") + } + + /** + * Converts a given Parquet `Type` into the corresponding + * [[org.apache.spark.sql.catalyst.types.DataType]]. + * + * Note that we apply the following conversion rules: + *
    + *
  • Primitive types are converter to the corresponding primitive type.
  • + *
  • Group types that have a single field with repetition `REPEATED` or themselves + * have repetition level `REPEATED` are converted to an [[ArrayType]] with the + * corresponding field type (possibly primitive) as element type.
  • + *
  • Other group types are converted as follows:
      + *
    • If they have a single field, they are converted into a [[StructType]] with + * the corresponding field type.
    • + *
    • If they have more than one field and repetition level `REPEATED` they are + * converted into an [[ArrayType]] with the corresponding [[StructType]] as complex + * element type.
    • + *
    • Otherwise they are converted into a [[StructType]] with the corresponding + * field types.
  • + *
+ * Note that fields are determined to be `nullable` if and only if their Parquet repetition + * level is not `REQUIRED`. + * + * @param parquetType The type to convert. + * @return The corresponding Catalyst type. + */ + def toDataType(parquetType: ParquetType): DataType = { + if (parquetType.isPrimitive) { + toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) + } + else { + val groupType = parquetType.asGroupType() + parquetType.getOriginalType match { + // if the schema was constructed programmatically there may be hints how to convert + // it inside the metadata via the OriginalType field + case ParquetOriginalType.LIST => { // TODO: check enums! + val fields = groupType.getFields.map { + field => new StructField(field.getName, toDataType(field), field.getRepetition != Repetition.REQUIRED) + } + if (fields.size == 1) { + new ArrayType(fields.apply(0).dataType) + } else { + new ArrayType(StructType(fields)) + } + } + case _ => { // everything else nested becomes a Struct, unless it has a single repeated field + // in which case it becomes an array (this should correspond to the inverse operation of + // parquet.schema.ConversionPatterns.listType) + if (groupType.getFieldCount == 1 && + (groupType.getFields.apply(0).getRepetition == Repetition.REPEATED || + groupType.getRepetition == Repetition.REPEATED)) { + val elementType = toDataType(groupType.getFields.apply(0)) + new ArrayType(elementType) + } else { + val fields = groupType + .getFields + .map(ptype => new StructField( + ptype.getName, + toDataType(ptype), + ptype.getRepetition != Repetition.REQUIRED)) + + if (groupType.getFieldCount == 1) { + new StructType(fields) + } else { + if (parquetType.getRepetition == Repetition.REPEATED) { + new ArrayType(StructType(fields)) + } else { + new StructType(fields) + } + } + } + } + } + } + } + + /** + * For a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] return + * the name of the corresponding Parquet primitive type or None if the given type + * is not primitive. + * + * @param ctype The type to convert + * @return The name of the corresponding Parquet primitive type + */ + def fromPrimitiveDataType(ctype: DataType): Option[ParquetPrimitiveTypeName] = ctype match { + case StringType => Some(ParquetPrimitiveTypeName.BINARY) + case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN) + case DoubleType => Some(ParquetPrimitiveTypeName.DOUBLE) + case ArrayType(ByteType) => Some(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) + case FloatType => Some(ParquetPrimitiveTypeName.FLOAT) + case IntegerType => Some(ParquetPrimitiveTypeName.INT32) + case LongType => Some(ParquetPrimitiveTypeName.INT64) + case _ => None + } + + /** + * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into + * the corrponsing Parquet `Type`. + * + * The conversion follows the rules below: + *
    + *
  • Primitive types are converted into Parquet's primitive types.
  • + *
  • [[org.apache.spark.sql.catalyst.types.StructType]]s are converted + * into Parquet's `GroupType` with the corresponding field types.
  • + *
  • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are handled as follows:
      + *
    • If their element is complex, that is of type + * [[org.apache.spark.sql.catalyst.types.StructType]], they are converted + * into a `GroupType` with the corresponding field types of the struct and + * original type of the `GroupType` is set to `LIST`.
    • + *
    • Otherwise, that is they contain a primitive they are converted into a `GroupType` + * that is also a list but has only a single field of the type corresponding to + * the element type.
  • + *
+ * Parquet's repetition level is set according to the following rule: + *
    + *
  • If the call to `fromDataType` is recursive inside an enclosing `ArrayType`, then + * the repetition level is set to `REPEATED`.
  • + *
  • Otherwise, if the attribute whose type is converted is `nullable`, the Parquet + * type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.
  • + *
+ * The single expection to this rule is an [[org.apache.spark.sql.catalyst.types.ArrayType]] + * that contains a [[org.apache.spark.sql.catalyst.types.StructType]], whose repetition level + * is always set to `REPEATED`. + * + @param ctype The type to convert. + * @param name The name of the [[org.apache.spark.sql.catalyst.expressions.Attribute]] whose type is converted + * @param nullable When true indicates that the attribute is nullable + * @param inArray When true indicates that this is a nested attribute inside an array. + * @return The corresponding Parquet type. + */ + def fromDataType( + ctype: DataType, + name: String, + nullable: Boolean = true, + inArray: Boolean = false): ParquetType = { + val repetition = + if (inArray) Repetition.REPEATED + else { + if (nullable) Repetition.OPTIONAL + else Repetition.REQUIRED + } + val primitiveType = fromPrimitiveDataType(ctype) + if (primitiveType.isDefined) { + new ParquetPrimitiveType(repetition, primitiveType.get, name) + } else { + ctype match { + case ArrayType(elementType: DataType) => { + elementType match { + case StructType(fields) => { // first case: array of structs + val parquetFieldTypes = fields.map( + f => fromDataType(f.dataType, f.name, f.nullable, inArray = false)) + assert(fields.size > 1, "Found struct inside array with a single field.. error parsing Catalyst schema") + new ParquetGroupType(Repetition.REPEATED, name, ParquetOriginalType.LIST, parquetFieldTypes) + } + case _ => { // second case: array of primitive types + val parquetElementType = fromDataType( + elementType, + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + nullable = false, + inArray = true) + ConversionPatterns.listType(repetition, name, parquetElementType) + } + } + } + // TODO: test structs inside arrays + case StructType(structFields) => { + val fields = structFields.map { + field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) + } + new ParquetGroupType(repetition, name, fields) + } + case _ => sys.error(s"Unsupported datatype $ctype") + } + } + } + + def consumeType( + consumer: RecordConsumer, + ctype: DataType, + record: Row, + index: Int): Unit = { + ctype match { + case StringType => consumer.addBinary( + Binary.fromByteArray( + record(index).asInstanceOf[String].getBytes("utf-8") + ) + ) + case IntegerType => consumer.addInteger(record.getInt(index)) + case LongType => consumer.addLong(record.getLong(index)) + case DoubleType => consumer.addDouble(record.getDouble(index)) + case FloatType => consumer.addFloat(record.getFloat(index)) + case BooleanType => consumer.addBoolean(record.getBoolean(index)) + case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer") + } + } + + def getSchema(schemaString: String) : MessageType = + MessageTypeParser.parseMessageType(schemaString) + + def convertToAttributes(parquetSchema: ParquetType): Seq[Attribute] = { + parquetSchema + .asGroupType() + .getFields + .map( + field => + new AttributeReference( + field.getName, + toDataType(field), + field.getRepetition != Repetition.REQUIRED)()) + } + + def convertFromAttributes(attributes: Seq[Attribute]): MessageType = { + val fields = attributes.map( + attribute => + fromDataType(attribute.dataType, attribute.name, attribute.nullable)) + new MessageType("root", fields) + } + + def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { + if (origPath == null) { + throw new IllegalArgumentException("Unable to write Parquet metadata: path is null") + } + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException( + s"Unable to write Parquet metadata: path $origPath is incorrectly formatted") + } + val path = origPath.makeQualified(fs) + if (fs.exists(path) && !fs.getFileStatus(path).isDir) { + throw new IllegalArgumentException(s"Expected to write to directory $path but found file") + } + val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) + if (fs.exists(metadataPath)) { + try { + fs.delete(metadataPath, true) + } catch { + case e: IOException => + throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE at $metadataPath") + } + } + val extraMetadata = new java.util.HashMap[String, String]() + extraMetadata.put("path", path.toString) + // TODO: add extra data, e.g., table name, date, etc.? + + val parquetSchema: MessageType = + ParquetTypesConverter.convertFromAttributes(attributes) + val metaData: FileMetaData = new FileMetaData( + parquetSchema, + extraMetadata, + "Spark") + + ParquetRelation.enableLogForwarding() + ParquetFileWriter.writeMetadataFile( + conf, + path, + new Footer(path, new ParquetMetadata(metaData, Nil)) :: Nil) + } + + /** + * Try to read Parquet metadata at the given Path. We first see if there is a summary file + * in the parent directory. If so, this is used. Else we read the actual footer at the given + * location. + * @param origPath The path at which we expect one (or more) Parquet files. + * @return The `ParquetMetadata` containing among other things the schema. + */ + def readMetaData(origPath: Path): ParquetMetadata = { + if (origPath == null) { + throw new IllegalArgumentException("Unable to read Parquet metadata: path is null") + } + val job = new Job() + // TODO: since this is called from ParquetRelation (LogicalPlan) we don't have access + // to SparkContext's hadoopConfig; in principle the default FileSystem may be different(?!) + val conf = ContextUtil.getConfiguration(job) + val fs: FileSystem = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException(s"Incorrectly formatted Parquet metadata path $origPath") + } + val path = origPath.makeQualified(fs) + if (!fs.getFileStatus(path).isDir) { + throw new IllegalArgumentException( + s"Expected $path for be a directory with Parquet files/metadata") + } + ParquetRelation.enableLogForwarding() + val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) + // if this is a new table that was just created we will find only the metadata file + if (fs.exists(metadataPath) && fs.isFile(metadataPath)) { + ParquetFileReader.readFooter(conf, metadataPath) + } else { + // there may be one or more Parquet files in the given directory + val footers = ParquetFileReader.readFooters(conf, fs.getFileStatus(path)) + // TODO: for now we assume that all footers (if there is more than one) have identical + // metadata; we may want to add a check here at some point + if (footers.size() == 0) { + throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path") + } + footers(0).getParquetMetadata + } + } +} From 4e25fcb420088b86e8f7cc7668b4d98d01c2fb4d Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Tue, 8 Apr 2014 10:27:26 +0300 Subject: [PATCH 13/48] Adding resolution of complex ArrayTypes --- .../apache/spark/sql/catalyst/SqlParser.scala | 3 +- .../catalyst/plans/logical/LogicalPlan.scala | 66 +++++++++++++++-- .../spark/sql/catalyst/types/dataTypes.scala | 6 +- .../spark/sql/parquet/ParquetTestData.scala | 43 ++++++++++- .../spark/sql/parquet/ParquetQuerySuite.scala | 71 +++++++++++++++---- 5 files changed, 167 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 46fcfbb9e26ba..d2baf09074799 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -90,7 +90,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { | failure("illegal character") ) - override def identChar = letter | elem('.') | elem('_') + override def identChar = letter | elem('.') | elem('_') | elem('[') | elem(']') override def whitespace: Parser[Any] = rep( whitespaceChar @@ -390,6 +390,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { FALSE ^^^ Literal(false, BooleanType) | cast | "(" ~> expression <~ ")" | + "[" ~> literal <~ "]" | function | "-" ~> literal ^^ UnaryMinus | ident ^^ UnresolvedAttribute | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 0933a31c362d8..99fa417eace83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.catalyst.types.{DataType, ArrayType, StructType} import org.apache.spark.sql.catalyst.trees abstract class LogicalPlan extends QueryPlan[LogicalPlan] { @@ -54,9 +54,41 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { /** * Optionally resolves the given string to a * [[catalyst.expressions.NamedExpression NamedExpression]]. The attribute is expressed as - * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. + * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. Fields + * can contain ordinal expressions, such as `field[i][j][k]...`. */ def resolve(name: String): Option[NamedExpression] = { + def expandFunc(expType: (Expression, DataType), field: String): (Expression, DataType) = { + val (exp, t) = expType + val ordinalRegExp = """(\[(\d+)\])""".r + val fieldName = if (field.matches("\\w*(\\[\\d\\])+")) { + field.substring(0, field.indexOf("[")) + } else { + field + } + t match { + case ArrayType(elementType) => + val ordinals = ordinalRegExp.findAllIn(field).matchData.map(_.group(2)) + (ordinals.foldLeft(exp)((v1: Expression, v2: String) => GetItem(v1, Literal(v2.toInt))), elementType) + case StructType(fields) => + // Note: this only works if we are not on the top-level! + val structField = fields.find(_.name == fieldName) + if (!structField.isDefined) { + throw new TreeNodeException( + this, s"Trying to resolve Attribute but field ${fieldName} is not defined") + } + structField.get.dataType match { + case ArrayType(elementType) => + val ordinals = ordinalRegExp.findAllIn(field).matchData.map(_.group(2)) + (ordinals.foldLeft(GetField(exp, fieldName).asInstanceOf[Expression])((v1: Expression, v2: String) => GetItem(v1, Literal(v2.toInt))), elementType) + case _ => + (GetField(exp, fieldName), structField.get.dataType) + } + case _ => + expType + } + } + val parts = name.split("\\.") // Collect all attributes that are output by this nodes children where either the first part // matches the name or where the first part matches the scope and the second part matches the @@ -67,16 +99,40 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { val remainingParts = if (option.qualifiers.contains(parts.head) && parts.size > 1) parts.drop(1) else parts if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil + // TODO from rebase! + /*val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts + val relevantRemaining = + if (remainingParts.head.matches("\\w*\\[(\\d+)\\]")) { // array field name + remainingParts.head.substring(0, remainingParts.head.indexOf("[")) + } else { + remainingParts.head + } + if (option.name == relevantRemaining) (option, remainingParts.tail.toList) :: Nil else Nil*/ } options.distinct match { - case (a, Nil) :: Nil => Some(a) // One match, no nested fields, use it. + case (a, Nil) :: Nil => { + a.dataType match { + case ArrayType(elementType) => + val expression = expandFunc((a: Expression, a.dataType), name)._1 + Some(Alias(expression, name)()) + case _ => Some(a) + } + } // One match, no nested fields, use it. // One match, but we also need to extract the requested nested field. case (a, nestedFields) :: Nil => a.dataType match { case StructType(fields) => - Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) - case _ => None // Don't know how to resolve these field references + // this is compatibility reasons with earlier code! TODO: why only nestedFields and not parts? + if ((parts(0) :: nestedFields).forall(!_.matches("\\w*\\[\\d+\\]+"))) { // not nested arrays, only fields + Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) + } else { + val expression = parts.foldLeft((a: Expression, a.dataType))(expandFunc)._1 + Some(Alias(expression, nestedFields.last)()) + } + case _ => + val expression = parts.foldLeft((a: Expression, a.dataType))(expandFunc)._1 + Some(Alias(expression, nestedFields.last)()) } case Nil => None // No matches. case ambiguousReferences => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 3b1c695911bf5..74cfa5962fe3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -29,11 +29,15 @@ abstract class DataType { case e: Expression if e.dataType == this => true case _ => false } + + def isPrimitive(): Boolean = false } case object NullType extends DataType -trait PrimitiveType +trait PrimitiveType extends DataType { + override def isPrimitive() = true +} abstract class NativeType extends DataType { type JvmType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 142c429240a55..d80c8890e7b58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -139,7 +139,7 @@ private[sql] object ParquetTestData { |optional group longs { |repeated int64 values; |} - |required group booleanNumberPairs { + |repeated group entries { |required double value; |optional boolean truth; |} @@ -153,8 +153,23 @@ private[sql] object ParquetTestData { |} """.stripMargin + val testNestedSchema3 = + """ + |message TestNested3 { + |required int32 x; + |repeated group booleanNumberPairs { + |required int32 key; + |repeated group value { + |required double nestedValue; + |optional boolean truth; + |} + |} + |} + """.stripMargin + val testNestedDir1 = Utils.createTempDir() val testNestedDir2 = Utils.createTempDir() + val testNestedDir3 = Utils.createTempDir() lazy val testNestedData1 = new ParquetRelation(testNestedDir1.toURI.toString) lazy val testNestedData2 = new ParquetRelation(testNestedDir2.toURI.toString) @@ -285,6 +300,32 @@ private[sql] object ParquetTestData { writer.close() } + def writeNestedFile3() { + testNestedDir3.delete() + val path: Path = testNestedDir3 + val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema3) + + val r1 = new SimpleGroup(schema) + r1.add(0, 1) + val g1 = r1.addGroup(1) + g1.add(0, 1) + val ng1 = g1.addGroup(1) + ng1.add(0, 1.5) + ng1.add(1, false) + val ng2 = g1.addGroup(1) + ng2.add(0, 2.5) + ng2.add(1, true) + val g2 = r1.addGroup(1) + g2.add(0, 2) + val ng3 = g2.addGroup(1) + ng3.add(0, 3.5) + ng3.add(1, false) + + val writeSupport = new TestGroupWriteSupport(schema) + val writer = new ParquetWriter[Group](path, writeSupport) + writer.write(r1) + writer.close() + } def readNestedFile(path: File, schemaString: String): Unit = { val configuration = new Configuration() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index e21d051ba30ae..88f48185006d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -34,6 +34,11 @@ import org.apache.spark.sql.SchemaRDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, DataType} +import org.apache.spark.sql.{parquet, SchemaRDD} +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import scala.Tuple2 +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute // Implicits import org.apache.spark.sql.test.TestSQLContext._ @@ -432,9 +437,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(0)(2)(0) === (1.toLong << 32)) assert(result(0)(2)(1) === (1.toLong << 33)) assert(result(0)(2)(2) === (1.toLong << 34)) - assert(result(0)(3).size === 2) - assert(result(0)(3)(0) === 2.5) - assert(result(0)(3)(1) === false) + assert(result(0)(3)(0).size === 2) + assert(result(0)(3)(0)(0) === 2.5) + assert(result(0)(3)(0)(1) === false) assert(result(0)(4).size === 2) assert(result(0)(4)(0).size === 2) assert(result(0)(4)(1).size === 1) @@ -452,23 +457,61 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(tmp(0)(0) === "Julien Le Dem") } + test("Projection in addressbook") { + implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] + val data = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir1.toString) + .toSchemaRDD + data.registerAsTable("data") + val tmp = sql("SELECT owner, contacts[1].name FROM data").collect() + assert(tmp.size === 2) + assert(tmp(0).size === 2) + assert(tmp(0)(0) === "Julien Le Dem") + assert(tmp(0)(1) === "Chris Aniszczyk") + assert(tmp(1)(0) === "A. Nonymous") + assert(tmp(1)(1) === null) + } + test("Simple query on nested int data") { implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD data.registerAsTable("data") - val tmp = sql("SELECT booleanNumberPairs.value, booleanNumberPairs.truth FROM data").collect() - assert(tmp(0)(0) === 2.5) - assert(tmp(0)(1) === false) - val result = sql("SELECT outerouter FROM data").collect() - // TODO: why does this not work? - //val result = sql("SELECT outerouter.values FROM data").collect() - // TODO: .. or this: - // val result = sql("SELECT outerouter[0] FROM data").collect() - assert(result(0)(0)(0)(0)(0) === 7) - assert(result(0)(0)(0)(1)(0) === 8) - assert(result(0)(0)(1)(0)(0) === 9) + val result1 = sql("SELECT entries[0].value FROM data").collect() + assert(result1.size === 1) + assert(result1(0).size === 1) + assert(result1(0)(0) === 2.5) + val result2 = sql("SELECT entries[0] FROM data").collect() + assert(result2.size === 1) + assert(result2(0)(0).size === 2) + assert(result2(0)(0)(0) === 2.5) + assert(result2(0)(0)(1) === false) + val result3 = sql("SELECT outerouter FROM data").collect() + assert(result3(0)(0)(0)(0)(0) === 7) + assert(result3(0)(0)(0)(1)(0) === 8) + assert(result3(0)(0)(1)(0)(0) === 9) + } + + test("nested structs") { + implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] + ParquetTestData.writeNestedFile3() + val data = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir3.toString) + .toSchemaRDD + data.registerAsTable("data") + val result1 = sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() + assert(result1.size === 1) + assert(result1(0).size === 1) + assert(result1(0)(0) === false) + val result2 = sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() + assert(result2.size === 1) + assert(result2(0).size === 1) + assert(result2(0)(0) === true) + val result3 = sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() + assert(result3.size === 1) + assert(result3(0).size === 1) + assert(result3(0)(0) === false) } /** From a594aed5423a31dab2e8dc09004a1b5d2657e146 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Tue, 8 Apr 2014 17:55:46 +0300 Subject: [PATCH 14/48] Scalastyle --- .../catalyst/plans/logical/LogicalPlan.scala | 14 +++++-- .../spark/sql/parquet/ParquetTypes.scala | 38 +++++++++++++------ 2 files changed, 36 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 99fa417eace83..b499436a49c18 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -69,7 +69,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { t match { case ArrayType(elementType) => val ordinals = ordinalRegExp.findAllIn(field).matchData.map(_.group(2)) - (ordinals.foldLeft(exp)((v1: Expression, v2: String) => GetItem(v1, Literal(v2.toInt))), elementType) + (ordinals.foldLeft(exp)((v1: Expression, v2: String) => + GetItem(v1, Literal(v2.toInt))), elementType) case StructType(fields) => // Note: this only works if we are not on the top-level! val structField = fields.find(_.name == fieldName) @@ -80,7 +81,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { structField.get.dataType match { case ArrayType(elementType) => val ordinals = ordinalRegExp.findAllIn(field).matchData.map(_.group(2)) - (ordinals.foldLeft(GetField(exp, fieldName).asInstanceOf[Expression])((v1: Expression, v2: String) => GetItem(v1, Literal(v2.toInt))), elementType) + (ordinals.foldLeft( + GetField(exp, fieldName).asInstanceOf[Expression])((v1: Expression, v2: String) => + GetItem(v1, Literal(v2.toInt))), + elementType) case _ => (GetField(exp, fieldName), structField.get.dataType) } @@ -123,8 +127,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { case (a, nestedFields) :: Nil => a.dataType match { case StructType(fields) => - // this is compatibility reasons with earlier code! TODO: why only nestedFields and not parts? - if ((parts(0) :: nestedFields).forall(!_.matches("\\w*\\[\\d+\\]+"))) { // not nested arrays, only fields + // this is compatibility reasons with earlier code! + // TODO: why only nestedFields and not parts? + // check for absence of nested arrays so there are only fields + if ((parts(0) :: nestedFields).forall(!_.matches("\\w*\\[\\d+\\]+"))) { Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) } else { val expression = parts.foldLeft((a: Expression, a.dataType))(expandFunc)._1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 04e4feffcc89b..7749f667e422c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -38,7 +38,8 @@ import org.apache.spark.sql.catalyst.types._ import scala.collection.JavaConversions._ private[parquet] object ParquetTypesConverter { - def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass + def isPrimitiveType(ctype: DataType): Boolean = + classOf[PrimitiveType] isAssignableFrom ctype.getClass def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { case ParquetPrimitiveTypeName.BINARY => StringType @@ -91,7 +92,10 @@ private[parquet] object ParquetTypesConverter { // it inside the metadata via the OriginalType field case ParquetOriginalType.LIST => { // TODO: check enums! val fields = groupType.getFields.map { - field => new StructField(field.getName, toDataType(field), field.getRepetition != Repetition.REQUIRED) + field => new StructField( + field.getName, + toDataType(field), + field.getRepetition != Repetition.REQUIRED) } if (fields.size == 1) { new ArrayType(fields.apply(0).dataType) @@ -99,7 +103,8 @@ private[parquet] object ParquetTypesConverter { new ArrayType(StructType(fields)) } } - case _ => { // everything else nested becomes a Struct, unless it has a single repeated field + case _ => { + // everything else nested becomes a Struct, unless it has a single repeated field // in which case it becomes an array (this should correspond to the inverse operation of // parquet.schema.ConversionPatterns.listType) if (groupType.getFieldCount == 1 && @@ -138,11 +143,13 @@ private[parquet] object ParquetTypesConverter { * @param ctype The type to convert * @return The name of the corresponding Parquet primitive type */ - def fromPrimitiveDataType(ctype: DataType): Option[ParquetPrimitiveTypeName] = ctype match { + def fromPrimitiveDataType(ctype: DataType): + Option[ParquetPrimitiveTypeName] = ctype match { case StringType => Some(ParquetPrimitiveTypeName.BINARY) case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN) case DoubleType => Some(ParquetPrimitiveTypeName.DOUBLE) - case ArrayType(ByteType) => Some(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) + case ArrayType(ByteType) => + Some(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) case FloatType => Some(ParquetPrimitiveTypeName.FLOAT) case IntegerType => Some(ParquetPrimitiveTypeName.INT32) case LongType => Some(ParquetPrimitiveTypeName.INT64) @@ -179,7 +186,8 @@ private[parquet] object ParquetTypesConverter { * is always set to `REPEATED`. * @param ctype The type to convert. - * @param name The name of the [[org.apache.spark.sql.catalyst.expressions.Attribute]] whose type is converted + * @param name The name of the [[org.apache.spark.sql.catalyst.expressions.Attribute]] + * whose type is converted * @param nullable When true indicates that the attribute is nullable * @param inArray When true indicates that this is a nested attribute inside an array. * @return The corresponding Parquet type. @@ -190,10 +198,10 @@ private[parquet] object ParquetTypesConverter { nullable: Boolean = true, inArray: Boolean = false): ParquetType = { val repetition = - if (inArray) Repetition.REPEATED - else { - if (nullable) Repetition.OPTIONAL - else Repetition.REQUIRED + if (inArray) { + Repetition.REPEATED + } else { + if (nullable) Repetition.OPTIONAL else Repetition.REQUIRED } val primitiveType = fromPrimitiveDataType(ctype) if (primitiveType.isDefined) { @@ -205,8 +213,14 @@ private[parquet] object ParquetTypesConverter { case StructType(fields) => { // first case: array of structs val parquetFieldTypes = fields.map( f => fromDataType(f.dataType, f.name, f.nullable, inArray = false)) - assert(fields.size > 1, "Found struct inside array with a single field.. error parsing Catalyst schema") - new ParquetGroupType(Repetition.REPEATED, name, ParquetOriginalType.LIST, parquetFieldTypes) + assert( + fields.size > 1, + "Found struct inside array with a single field.. error parsing Catalyst schema") + new ParquetGroupType( + Repetition.REPEATED, + name, + ParquetOriginalType.LIST, + parquetFieldTypes) } case _ => { // second case: array of primitive types val parquetElementType = fromDataType( From b539fdeabfe6beb178acadb55c8a6339af4f06c3 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 11 Apr 2014 17:54:04 +0300 Subject: [PATCH 15/48] First commit for MapType --- .../spark/sql/parquet/ParquetConverter.scala | 72 +++++++++++++++- .../spark/sql/parquet/ParquetTestData.scala | 54 ++++++++++++ .../spark/sql/parquet/ParquetTypes.scala | 86 ++++++++++++++----- .../spark/sql/parquet/ParquetQuerySuite.scala | 47 +++++++++- 4 files changed, 236 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 2eee9a4c30cf7..94a7ab719789f 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 @@ -26,6 +26,7 @@ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} import org.apache.spark.sql.parquet.CatalystConverter.FieldType +import scala.collection.mutable private[parquet] object CatalystConverter { // The type internally used for fields @@ -55,6 +56,14 @@ private[parquet] object CatalystConverter { case StructType(fields: Seq[StructField]) => { new CatalystStructConverter(fields, fieldIndex, parent) } + case MapType(keyType: DataType, valueType: DataType) => { + new CatalystMapConverter( + Seq( + new FieldType("key", keyType, false), + new FieldType("value", valueType, true)), + fieldIndex, + parent) + } case ctype: NativeType => { // note: for some reason matching for StringType fails so use this ugly if instead if (ctype == StringType) { @@ -396,6 +405,67 @@ private[parquet] class CatalystStructConverter( override def getCurrentRecord: Row = throw new UnsupportedOperationException } -// TODO: add MapConverter +private[parquet] class CatalystMapConverter( + protected[parquet] val schema: Seq[FieldType], + override protected[parquet] val index: Int, + override protected[parquet] val parent: CatalystConverter) + extends GroupConverter with CatalystConverter { + + private val map = new mutable.HashMap[Any, Any]() + + private val keyValueConverter = new GroupConverter with 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 def getCurrentRecord: Row = throw new UnsupportedOperationException + } + + override protected[parquet] val size: Int = 1 + + override protected[parquet] def clearBuffer(): Unit = {} + + override def start(): Unit = { + map.clear() + } + + // TODO: think about reusing the buffer + override def end(): Unit = { + assert(!isRootConverter) + parent.updateField(index, map) + } + + override def getConverter(fieldIndex: Int): Converter = keyValueConverter + + override def getCurrentRecord: Row = throw new UnsupportedOperationException + + 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/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index d80c8890e7b58..c0ae738418202 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -167,9 +167,32 @@ private[sql] object ParquetTestData { |} """.stripMargin + val testNestedSchema4 = + """ + |message TestNested4 { + |required int32 x; + |optional group data1 { + |repeated group map { + |required binary key; + |optional int32 value; + |} + |} + |required group data2 { + |repeated group map { + |required int32 key; + |optional group value { + |required int64 payload1; + |optional binary payload2; + |} + |} + |} + |} + """.stripMargin + val testNestedDir1 = Utils.createTempDir() val testNestedDir2 = Utils.createTempDir() val testNestedDir3 = Utils.createTempDir() + val testNestedDir4 = Utils.createTempDir() lazy val testNestedData1 = new ParquetRelation(testNestedDir1.toURI.toString) lazy val testNestedData2 = new ParquetRelation(testNestedDir2.toURI.toString) @@ -327,6 +350,37 @@ private[sql] object ParquetTestData { writer.close() } + def writeNestedFile4() { + testNestedDir4.delete() + val path: Path = testNestedDir4 + val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema4) + + val r1 = new SimpleGroup(schema) + r1.add(0, 7) + val map1 = r1.addGroup(1) + val keyValue1 = map1.addGroup(0) + keyValue1.add(0, "key1") + keyValue1.add(1, 1) + val keyValue2 = map1.addGroup(0) + keyValue2.add(0, "key2") + keyValue2.add(1, 2) + val map2 = r1.addGroup(2) + val keyValue3 = map2.addGroup(0) + keyValue3.add(0, 7) + val valueGroup1 = keyValue3.addGroup(1) + valueGroup1.add(0, 42.toLong) + valueGroup1.add(1, "the answer") + val keyValue4 = map2.addGroup(0) + keyValue4.add(0, 8) + val valueGroup2 = keyValue4.addGroup(1) + valueGroup2.add(0, 49.toLong) + + val writeSupport = new TestGroupWriteSupport(schema) + val writer = new ParquetWriter[Group](path, writeSupport) + writer.write(r1) + writer.close() + } + def readNestedFile(path: File, schemaString: String): Unit = { val configuration = new Configuration() val fs: FileSystem = path.getFileSystem(configuration) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 7749f667e422c..0318055eb4512 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -63,13 +63,17 @@ private[parquet] object ParquetTypesConverter { * Note that we apply the following conversion rules: *
    *
  • Primitive types are converter to the corresponding primitive type.
  • - *
  • Group types that have a single field with repetition `REPEATED` or themselves - * have repetition level `REPEATED` are converted to an [[ArrayType]] with the - * corresponding field type (possibly primitive) as element type.
  • + *
  • Group types that have a single field that is itself a group, which has repetition + * level `REPEATED` and two fields (named `key` and `value`), are converted to + * a [[MapType]] with the corresponding key and value (value possibly complex) + * as element type.
  • *
  • Other group types are converted as follows:
      - *
    • If they have a single field, they are converted into a [[StructType]] with + *
    • Group types that have a single field with repetition `REPEATED` or themselves + * have repetition level `REPEATED` are converted to an [[ArrayType]] with the + * corresponding field type (possibly primitive) as element type.
    • + *
    • Other groups with a single field are converted into a [[StructType]] with * the corresponding field type.
    • - *
    • If they have more than one field and repetition level `REPEATED` they are + *
    • If groups have more than one field and repetition level `REPEATED` they are * converted into an [[ArrayType]] with the corresponding [[StructType]] as complex * element type.
    • *
    • Otherwise they are converted into a [[StructType]] with the corresponding @@ -82,16 +86,33 @@ private[parquet] object ParquetTypesConverter { * @return The corresponding Catalyst type. */ def toDataType(parquetType: ParquetType): DataType = { + def correspondsToMap(groupType: ParquetGroupType): Boolean = { + if (groupType.getFieldCount != 1 || groupType.getFields.apply(0).isPrimitive) { + false + } else { + // This mostly follows the convention in ``parquet.schema.ConversionPatterns`` + val keyValueGroup = groupType.getFields.apply(0).asGroupType() + keyValueGroup.getRepetition == Repetition.REPEATED && + keyValueGroup.getName == "map" && + keyValueGroup.getFields.apply(0).getName == "key" && + keyValueGroup.getFields.apply(1).getName == "value" + } + } + def correspondsToArray(groupType: ParquetGroupType): Boolean = { + groupType.getFieldCount == 1 && + (groupType.getFields.apply(0).getRepetition == Repetition.REPEATED || + groupType.getRepetition == Repetition.REPEATED) + } + if (parquetType.isPrimitive) { toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) - } - else { + } else { val groupType = parquetType.asGroupType() parquetType.getOriginalType match { // if the schema was constructed programmatically there may be hints how to convert // it inside the metadata via the OriginalType field case ParquetOriginalType.LIST => { // TODO: check enums! - val fields = groupType.getFields.map { + val fields = groupType.getFields.map { field => new StructField( field.getName, toDataType(field), @@ -103,16 +124,29 @@ private[parquet] object ParquetTypesConverter { new ArrayType(StructType(fields)) } } + case ParquetOriginalType.MAP => { + assert( + !groupType.getFields.apply(0).isPrimitive, + "Parquet Map type malformatted: expected nested group for map!") + val keyValueGroup = groupType.getFields.apply(0).asGroupType() + assert( + keyValueGroup.getFieldCount == 2, + "Parquet Map type malformatted: nested group should have 2 (key, value) fields!") + val keyType = toDataType(keyValueGroup.getFields.apply(0)) + val valueType = toDataType(keyValueGroup.getFields.apply(1)) + new MapType(keyType, valueType) + } case _ => { - // everything else nested becomes a Struct, unless it has a single repeated field - // in which case it becomes an array (this should correspond to the inverse operation of - // parquet.schema.ConversionPatterns.listType) - if (groupType.getFieldCount == 1 && - (groupType.getFields.apply(0).getRepetition == Repetition.REPEATED || - groupType.getRepetition == Repetition.REPEATED)) { + // Note: the order of these checks is important! + if (correspondsToMap(groupType)) { // MapType + val keyValueGroup = groupType.getFields.apply(0).asGroupType() + val keyType = toDataType(keyValueGroup.getFields.apply(0)) + val valueType = toDataType(keyValueGroup.getFields.apply(1)) + new MapType(keyType, valueType) + } else if (correspondsToArray(groupType)) { // ArrayType val elementType = toDataType(groupType.getFields.apply(0)) new ArrayType(elementType) - } else { + } else { // everything else: StructType val fields = groupType .getFields .map(ptype => new StructField( @@ -164,7 +198,10 @@ private[parquet] object ParquetTypesConverter { *
        *
      • Primitive types are converted into Parquet's primitive types.
      • *
      • [[org.apache.spark.sql.catalyst.types.StructType]]s are converted - * into Parquet's `GroupType` with the corresponding field types.
      • + * into Parquet's `GroupType` with the corresponding field types. + *
      • [[org.apache.spark.sql.catalyst.types.MapType]]s are converted + * into a nested (2-level) Parquet `GroupType` with two fields: a key type and + * a value type. The nested group has repetition level `REPEATED`.
      • *
      • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are handled as follows:
          *
        • If their element is complex, that is of type * [[org.apache.spark.sql.catalyst.types.StructType]], they are converted @@ -174,18 +211,18 @@ private[parquet] object ParquetTypesConverter { * that is also a list but has only a single field of the type corresponding to * the element type.
      • *
      - * Parquet's repetition level is set according to the following rule: + * Parquet's repetition level is generally set according to the following rule: *
        - *
      • If the call to `fromDataType` is recursive inside an enclosing `ArrayType`, then - * the repetition level is set to `REPEATED`.
      • + *
      • If the call to `fromDataType` is recursive inside an enclosing `ArrayType` or + * `MapType`, then the repetition level is set to `REPEATED`.
      • *
      • Otherwise, if the attribute whose type is converted is `nullable`, the Parquet * type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.
      • *
      - * The single expection to this rule is an [[org.apache.spark.sql.catalyst.types.ArrayType]] + * The single exception to this rule is an [[org.apache.spark.sql.catalyst.types.ArrayType]] * that contains a [[org.apache.spark.sql.catalyst.types.StructType]], whose repetition level * is always set to `REPEATED`. * - @param ctype The type to convert. + * @param ctype The type to convert. * @param name The name of the [[org.apache.spark.sql.catalyst.expressions.Attribute]] * whose type is converted * @param nullable When true indicates that the attribute is nullable @@ -239,6 +276,13 @@ private[parquet] object ParquetTypesConverter { } new ParquetGroupType(repetition, name, fields) } + case MapType(keyType, valueType) => { + ConversionPatterns.mapType( + repetition, + name, + fromDataType(keyType, "key", false, inArray = false), + fromDataType(valueType, "value", true, inArray = false)) + } case _ => sys.error(s"Unsupported datatype $ctype") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 88f48185006d9..4dcf6d472bd76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -74,6 +74,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA ParquetTestData.writeFilterFile() ParquetTestData.writeNestedFile1() ParquetTestData.writeNestedFile2() + ParquetTestData.writeNestedFile3() + ParquetTestData.writeNestedFile4() testRDD = parquetFile(ParquetTestData.testDir.toString) testRDD.registerAsTable("testsource") parquetFile(ParquetTestData.testFilterDir.toString) @@ -85,6 +87,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(ParquetTestData.testFilterDir) Utils.deleteRecursively(ParquetTestData.testNestedDir1) Utils.deleteRecursively(ParquetTestData.testNestedDir2) + Utils.deleteRecursively(ParquetTestData.testNestedDir3) + Utils.deleteRecursively(ParquetTestData.testNestedDir4) // here we should also unregister the table?? } @@ -495,7 +499,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("nested structs") { implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] - ParquetTestData.writeNestedFile3() val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir3.toString) .toSchemaRDD @@ -514,6 +517,48 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result3(0)(0) === false) } + test("simple map") { + implicit def anyToMap(value: Any) = value.asInstanceOf[collection.mutable.HashMap[String, Int]] + val data = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir4.toString) + .toSchemaRDD + data.registerAsTable("mapTable") + val result1 = sql("SELECT data1 FROM mapTable").collect() + assert(result1.size === 1) + assert(result1(0)(0).toMap.getOrElse("key1", 0) === 1) + assert(result1(0)(0).toMap.getOrElse("key2", 0) === 2) + } + + test("map with struct values") { + //implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] + implicit def anyToMap(value: Any) = value.asInstanceOf[collection.mutable.HashMap[Int, Row]] + //val data = TestSQLContext + // .parquetFile(ParquetTestData.testNestedDir4.toString) + // .toSchemaRDD + val data = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir4.toString) + .toSchemaRDD + data.registerAsTable("mapTable") + + /*ParquetTestData.readNestedFile( + ParquetTestData.testNestedDir4, + ParquetTestData.testNestedSchema4) + val result = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir4.toString) + .toSchemaRDD + .collect()*/ + val result1 = sql("SELECT data2 FROM mapTable").collect() + assert(result1.size === 1) + val entry1 = result1(0)(0).getOrElse(7, null) + assert(entry1 != null) + assert(entry1(0) === 42) + assert(entry1(1) === "the answer") + val entry2 = result1(0)(0).getOrElse(8, null) + assert(entry2 != null) + assert(entry2(0) === 49) + assert(entry2(1) === null) + } + /** * Creates an empty SchemaRDD backed by a ParquetRelation. * From 824500c912b4f37e8d4b690ffe7d3c0c3572e6a6 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 11 Apr 2014 21:25:47 +0300 Subject: [PATCH 16/48] Adding attribute resolution for MapType --- .../catalyst/plans/logical/LogicalPlan.scala | 38 ++++++++++++++----- .../spark/sql/parquet/ParquetConverter.scala | 3 +- .../spark/sql/parquet/ParquetTestData.scala | 8 ++-- .../spark/sql/parquet/ParquetQuerySuite.scala | 26 +++++-------- 4 files changed, 45 insertions(+), 30 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index b499436a49c18..1f814c560fd64 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -20,8 +20,16 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.types.{DataType, ArrayType, StructType} +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees +import scala.util.matching.Regex +import org.apache.spark.sql.catalyst.types.ArrayType +import org.apache.spark.sql.catalyst.expressions.GetField +import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.catalyst.types.MapType +import scala.Some +import org.apache.spark.sql.catalyst.expressions.Alias +import org.apache.spark.sql.catalyst.expressions.GetItem abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => @@ -60,20 +68,32 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { def resolve(name: String): Option[NamedExpression] = { def expandFunc(expType: (Expression, DataType), field: String): (Expression, DataType) = { val (exp, t) = expType - val ordinalRegExp = """(\[(\d+)\])""".r - val fieldName = if (field.matches("\\w*(\\[\\d\\])+")) { + val ordinalRegExp = """(\[(\d+|\w+)\])""".r + val fieldName = if (ordinalRegExp.findFirstIn(field).isDefined) { field.substring(0, field.indexOf("[")) } else { field } t match { case ArrayType(elementType) => - val ordinals = ordinalRegExp.findAllIn(field).matchData.map(_.group(2)) + val ordinals = ordinalRegExp + .findAllIn(field) + .matchData + .map(_.group(2)) (ordinals.foldLeft(exp)((v1: Expression, v2: String) => GetItem(v1, Literal(v2.toInt))), elementType) + case MapType(keyType, valueType) => + val ordinals = ordinalRegExp + .findAllIn(field) + .matchData + .map(_.group(2)) + // TODO: we should recover the JVM type of valueType to match the + // actual type of the key?! should we restrict ourselves to NativeType? + (ordinals.foldLeft(exp)((v1: Expression, v2: String) => + GetItem(v1, Literal(v2, keyType))), valueType) case StructType(fields) => - // Note: this only works if we are not on the top-level! - val structField = fields.find(_.name == fieldName) + val structField = fields + .find(_.name == fieldName) if (!structField.isDefined) { throw new TreeNodeException( this, s"Trying to resolve Attribute but field ${fieldName} is not defined") @@ -106,7 +126,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { // TODO from rebase! /*val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts val relevantRemaining = - if (remainingParts.head.matches("\\w*\\[(\\d+)\\]")) { // array field name + if (remainingParts.head.matches("\\w*\\[(\\d+|\\w+)\\]")) { // array field name remainingParts.head.substring(0, remainingParts.head.indexOf("[")) } else { remainingParts.head @@ -117,7 +137,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { options.distinct match { case (a, Nil) :: Nil => { a.dataType match { - case ArrayType(elementType) => + case ArrayType(_) | MapType(_, _) => val expression = expandFunc((a: Expression, a.dataType), name)._1 Some(Alias(expression, name)()) case _ => Some(a) @@ -130,7 +150,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { // this is compatibility reasons with earlier code! // TODO: why only nestedFields and not parts? // check for absence of nested arrays so there are only fields - if ((parts(0) :: nestedFields).forall(!_.matches("\\w*\\[\\d+\\]+"))) { + if ((parts(0) :: nestedFields).forall(!_.matches("\\w*\\[(\\d+|\\w+)\\]+"))) { Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) } else { val expression = parts.foldLeft((a: Expression, a.dataType))(expandFunc)._1 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 94a7ab719789f..37306b4a26078 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 @@ -455,8 +455,7 @@ private[parquet] class CatalystMapConverter( // TODO: think about reusing the buffer override def end(): Unit = { - assert(!isRootConverter) - parent.updateField(index, map) + parent.updateField(index, map.toMap) } override def getConverter(fieldIndex: Int): Converter = keyValueConverter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index c0ae738418202..e3957732eb341 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -179,7 +179,7 @@ private[sql] object ParquetTestData { |} |required group data2 { |repeated group map { - |required int32 key; + |required binary key; |optional group value { |required int64 payload1; |optional binary payload2; @@ -366,12 +366,14 @@ private[sql] object ParquetTestData { keyValue2.add(1, 2) val map2 = r1.addGroup(2) val keyValue3 = map2.addGroup(0) - keyValue3.add(0, 7) + // TODO: currently only string key type supported + keyValue3.add(0, "7") val valueGroup1 = keyValue3.addGroup(1) valueGroup1.add(0, 42.toLong) valueGroup1.add(1, "the answer") val keyValue4 = map2.addGroup(0) - keyValue4.add(0, 8) + // TODO: currently only string key type supported + keyValue4.add(0, "8") val valueGroup2 = keyValue4.addGroup(1) valueGroup2.add(0, 49.toLong) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 4dcf6d472bd76..ea940184ca4e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -518,7 +518,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("simple map") { - implicit def anyToMap(value: Any) = value.asInstanceOf[collection.mutable.HashMap[String, Int]] + implicit def anyToMap(value: Any) = value.asInstanceOf[Map[String, Int]] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD @@ -527,36 +527,30 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result1.size === 1) assert(result1(0)(0).toMap.getOrElse("key1", 0) === 1) assert(result1(0)(0).toMap.getOrElse("key2", 0) === 2) + val result2 = sql("SELECT data1[key1] FROM mapTable").collect() + assert(result2(0)(0) === 1) } test("map with struct values") { - //implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] - implicit def anyToMap(value: Any) = value.asInstanceOf[collection.mutable.HashMap[Int, Row]] - //val data = TestSQLContext - // .parquetFile(ParquetTestData.testNestedDir4.toString) - // .toSchemaRDD + implicit def anyToMap(value: Any) = value.asInstanceOf[Map[String, Row]] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD data.registerAsTable("mapTable") - - /*ParquetTestData.readNestedFile( - ParquetTestData.testNestedDir4, - ParquetTestData.testNestedSchema4) - val result = TestSQLContext - .parquetFile(ParquetTestData.testNestedDir4.toString) - .toSchemaRDD - .collect()*/ val result1 = sql("SELECT data2 FROM mapTable").collect() assert(result1.size === 1) - val entry1 = result1(0)(0).getOrElse(7, null) + val entry1 = result1(0)(0).getOrElse("7", null) assert(entry1 != null) assert(entry1(0) === 42) assert(entry1(1) === "the answer") - val entry2 = result1(0)(0).getOrElse(8, null) + val entry2 = result1(0)(0).getOrElse("8", null) assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) + val result2 = sql("SELECT data2[7].payload1, data2[7].payload2 FROM mapTable").collect() + assert(result2.size === 1) + assert(result2(0)(0) === 42.toLong) + assert(result2(0)(1) === "the answer") } /** From f777b4b772e595bb0eceb4db71125d64f3bf2712 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 11 Apr 2014 21:28:51 +0300 Subject: [PATCH 17/48] Scalastyle --- .../spark/sql/parquet/ParquetConverter.scala | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 37306b4a26078..1bec48204fc2b 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 @@ -419,7 +419,9 @@ private[parquet] class CatalystMapConverter( 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 getConverter(fieldIndex: Int): Converter = { + if (fieldIndex == 0) keyConverter else valueConverter + } override def end(): Unit = CatalystMapConverter.this.map += currentKey -> currentValue @@ -432,13 +434,15 @@ private[parquet] class CatalystMapConverter( 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 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 = {} From d1911dcc19a7a3c5a83823a4938f1ce0bbe3efbd Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 12 Apr 2014 19:25:55 +0300 Subject: [PATCH 18/48] Simplifying ArrayType conversion --- .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 20 +-- .../spark/sql/parquet/ParquetTestData.scala | 56 +++++--- .../spark/sql/parquet/ParquetTypes.scala | 131 +++++++----------- 4 files changed, 92 insertions(+), 117 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 1f814c560fd64..906339c73d95a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -87,7 +87,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { .findAllIn(field) .matchData .map(_.group(2)) - // TODO: we should recover the JVM type of valueType to match the + // TODO: we should recover the JVM type of keyType to match the // actual type of the key?! should we restrict ourselves to NativeType? (ordinals.foldLeft(exp)((v1: Expression, v2: String) => GetItem(v1, Literal(v2, keyType))), valueType) 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 1bec48204fc2b..3cb25925c7b08 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 @@ -32,12 +32,11 @@ private[parquet] object CatalystConverter { // The type internally used for fields type FieldType = StructField - // Note: repeated primitive fields that form an array (together with - // their surrounding group) need to have this name in the schema - // TODO: "values" is a generic name but without it the Parquet column path would - // be incomplete and values may be silently dropped; better would be to give - // primitive-type array elements a name of some sort + // This is mostly Parquet convention (see, e.g., `ConversionPatterns`) val ARRAY_ELEMENTS_SCHEMA_NAME = "values" + val MAP_KEY_SCHEMA_NAME = "key" + val MAP_VALUE_SCHEMA_NAME = "value" + val MAP_SCHEMA_NAME = "map" protected[parquet] def createConverter( field: FieldType, @@ -46,12 +45,7 @@ private[parquet] object CatalystConverter { val fieldType: DataType = field.dataType fieldType match { case ArrayType(elementType: DataType) => { - elementType match { - case StructType(fields) => - if (fields.size > 1) new CatalystGroupConverter(fields, fieldIndex, parent) - else new CatalystArrayConverter(fields(0).dataType, fieldIndex, parent) - case _ => new CatalystArrayConverter(elementType, fieldIndex, parent) - } + new CatalystArrayConverter(elementType, fieldIndex, parent) } case StructType(fields: Seq[StructField]) => { new CatalystStructConverter(fields, fieldIndex, parent) @@ -59,8 +53,8 @@ private[parquet] object CatalystConverter { case MapType(keyType: DataType, valueType: DataType) => { new CatalystMapConverter( Seq( - new FieldType("key", keyType, false), - new FieldType("value", valueType, true)), + new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false), + new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)), fieldIndex, parent) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index e3957732eb341..8a34ee3b92eff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -123,9 +123,11 @@ private[sql] object ParquetTestData { |optional group ownerPhoneNumbers { |repeated binary values; |} - |repeated group contacts { - |required binary name; - |optional binary phoneNumber; + |optional group contacts { + |repeated group values { + |required binary name; + |optional binary phoneNumber; + |} |} |} """.stripMargin @@ -139,9 +141,11 @@ private[sql] object ParquetTestData { |optional group longs { |repeated int64 values; |} - |repeated group entries { - |required double value; - |optional boolean truth; + |required group entries { + |repeated group values { + |required double value; + |optional boolean truth; + |} |} |optional group outerouter { |repeated group values { @@ -156,12 +160,16 @@ private[sql] object ParquetTestData { val testNestedSchema3 = """ |message TestNested3 { - |required int32 x; - |repeated group booleanNumberPairs { - |required int32 key; - |repeated group value { - |required double nestedValue; - |optional boolean truth; + |required int32 x; + |optional group booleanNumberPairs { + |repeated group values { + |required int32 key; + |optional group value { + |repeated group values { + |required double nestedValue; + |optional boolean truth; + |} + |} |} |} |} @@ -268,12 +276,11 @@ private[sql] object ParquetTestData { .append("values", "555 123 4567") .append("values", "555 666 1337") .append("values", "XXX XXX XXXX") - r1.addGroup(2) - // .addGroup(0) + val contacts = r1.addGroup(2) + contacts.addGroup(0) .append("name", "Dmitriy Ryaboy") .append("phoneNumber", "555 987 6543") - r1.addGroup(2) - // .addGroup(0) + contacts.addGroup(0) .append("name", "Chris Aniszczyk") val r2 = new SimpleGroup(schema) @@ -298,9 +305,9 @@ private[sql] object ParquetTestData { longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME , 1.toLong << 32) longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 1.toLong << 33) longs.add(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 1.toLong << 34) - val booleanNumberPairs = r1.addGroup(3) - booleanNumberPairs.add("value", 2.5) - booleanNumberPairs.add("truth", false) + val booleanNumberPair = r1.addGroup(3).addGroup(0) + booleanNumberPair.add("value", 2.5) + booleanNumberPair.add("truth", false) val top_level = r1.addGroup(4) val second_level_a = top_level.addGroup(0) val second_level_b = top_level.addGroup(0) @@ -330,17 +337,20 @@ private[sql] object ParquetTestData { val r1 = new SimpleGroup(schema) r1.add(0, 1) - val g1 = r1.addGroup(1) + val booleanNumberPairs = r1.addGroup(1) + val g1 = booleanNumberPairs.addGroup(0) g1.add(0, 1) - val ng1 = g1.addGroup(1) + val nested1 = g1.addGroup(1) + val ng1 = nested1.addGroup(0) ng1.add(0, 1.5) ng1.add(1, false) - val ng2 = g1.addGroup(1) + val ng2 = nested1.addGroup(0) ng2.add(0, 2.5) ng2.add(1, true) - val g2 = r1.addGroup(1) + val g2 = booleanNumberPairs.addGroup(0) g2.add(0, 2) val ng3 = g2.addGroup(1) + .addGroup(0) ng3.add(0, 3.5) ng3.add(1, false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 0318055eb4512..867dc505e095a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -64,20 +64,15 @@ private[parquet] object ParquetTypesConverter { *
        *
      • Primitive types are converter to the corresponding primitive type.
      • *
      • Group types that have a single field that is itself a group, which has repetition - * level `REPEATED` and two fields (named `key` and `value`), are converted to - * a [[MapType]] with the corresponding key and value (value possibly complex) - * as element type.
      • - *
      • Other group types are converted as follows:
          - *
        • Group types that have a single field with repetition `REPEATED` or themselves - * have repetition level `REPEATED` are converted to an [[ArrayType]] with the - * corresponding field type (possibly primitive) as element type.
        • - *
        • Other groups with a single field are converted into a [[StructType]] with - * the corresponding field type.
        • - *
        • If groups have more than one field and repetition level `REPEATED` they are - * converted into an [[ArrayType]] with the corresponding [[StructType]] as complex - * element type.
        • - *
        • Otherwise they are converted into a [[StructType]] with the corresponding - * field types.
      • + * level `REPEATED` are treated as follows:
          + *
        • If the nested group has name `values` and repetition level `REPEATED`, the + * surrounding group is converted into an [[ArrayType]] with the + * corresponding field type (primitive or complex) as element type.
        • + *
        • If the nested group has name `map`, repetition level `REPEATED` and two fields + * (named `key` and `value`), the surrounding group is converted into a [[MapType]] + * with the corresponding key and value (value possibly complex) types.
        • + *
        • Other group types are converted into a [[StructType]] with the corresponding + * field types.
        *
      * Note that fields are determined to be `nullable` if and only if their Parquet repetition * level is not `REQUIRED`. @@ -93,15 +88,16 @@ private[parquet] object ParquetTypesConverter { // This mostly follows the convention in ``parquet.schema.ConversionPatterns`` val keyValueGroup = groupType.getFields.apply(0).asGroupType() keyValueGroup.getRepetition == Repetition.REPEATED && - keyValueGroup.getName == "map" && - keyValueGroup.getFields.apply(0).getName == "key" && - keyValueGroup.getFields.apply(1).getName == "value" + keyValueGroup.getName == CatalystConverter.MAP_SCHEMA_NAME && + keyValueGroup.getFieldCount == 2 && + keyValueGroup.getFields.apply(0).getName == CatalystConverter.MAP_KEY_SCHEMA_NAME && + keyValueGroup.getFields.apply(1).getName == CatalystConverter.MAP_VALUE_SCHEMA_NAME } } def correspondsToArray(groupType: ParquetGroupType): Boolean = { groupType.getFieldCount == 1 && - (groupType.getFields.apply(0).getRepetition == Repetition.REPEATED || - groupType.getRepetition == Repetition.REPEATED) + groupType.getFieldName(0) == CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME && + groupType.getFields.apply(0).getRepetition == Repetition.REPEATED } if (parquetType.isPrimitive) { @@ -112,17 +108,9 @@ private[parquet] object ParquetTypesConverter { // if the schema was constructed programmatically there may be hints how to convert // it inside the metadata via the OriginalType field case ParquetOriginalType.LIST => { // TODO: check enums! - val fields = groupType.getFields.map { - field => new StructField( - field.getName, - toDataType(field), - field.getRepetition != Repetition.REQUIRED) - } - if (fields.size == 1) { - new ArrayType(fields.apply(0).dataType) - } else { - new ArrayType(StructType(fields)) - } + assert(groupType.getFieldCount == 1) + val field = groupType.getFields.apply(0) + new ArrayType(toDataType(field)) } case ParquetOriginalType.MAP => { assert( @@ -153,16 +141,7 @@ private[parquet] object ParquetTypesConverter { ptype.getName, toDataType(ptype), ptype.getRepetition != Repetition.REQUIRED)) - - if (groupType.getFieldCount == 1) { - new StructType(fields) - } else { - if (parquetType.getRepetition == Repetition.REPEATED) { - new ArrayType(StructType(fields)) - } else { - new StructType(fields) - } - } + new StructType(fields) } } } @@ -199,17 +178,17 @@ private[parquet] object ParquetTypesConverter { *
    • Primitive types are converted into Parquet's primitive types.
    • *
    • [[org.apache.spark.sql.catalyst.types.StructType]]s are converted * into Parquet's `GroupType` with the corresponding field types.
    • + *
    • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are converterd + * into a 2-level nested group, where the outer group has the inner + * group as sole field. The inner group has name `values` and + * repetition level `REPEATED` and has the element type of + * the array as schema. We use Parquet's `ConversionPatterns` for this + * purpose.
    • *
    • [[org.apache.spark.sql.catalyst.types.MapType]]s are converted - * into a nested (2-level) Parquet `GroupType` with two fields: a key type and - * a value type. The nested group has repetition level `REPEATED`.
    • - *
    • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are handled as follows:
        - *
      • If their element is complex, that is of type - * [[org.apache.spark.sql.catalyst.types.StructType]], they are converted - * into a `GroupType` with the corresponding field types of the struct and - * original type of the `GroupType` is set to `LIST`.
      • - *
      • Otherwise, that is they contain a primitive they are converted into a `GroupType` - * that is also a list but has only a single field of the type corresponding to - * the element type.
    • + * into a nested (2-level) Parquet `GroupType` with two fields: a key + * type and a value type. The nested group has repetition level + * `REPEATED` and name `map`. We use Parquet's `ConversionPatterns` + * for this purpose *
    * Parquet's repetition level is generally set according to the following rule: *
      @@ -218,11 +197,8 @@ private[parquet] object ParquetTypesConverter { *
    • Otherwise, if the attribute whose type is converted is `nullable`, the Parquet * type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.
    • *
    - * The single exception to this rule is an [[org.apache.spark.sql.catalyst.types.ArrayType]] - * that contains a [[org.apache.spark.sql.catalyst.types.StructType]], whose repetition level - * is always set to `REPEATED`. * - * @param ctype The type to convert. + *@param ctype The type to convert * @param name The name of the [[org.apache.spark.sql.catalyst.expressions.Attribute]] * whose type is converted * @param nullable When true indicates that the attribute is nullable @@ -245,31 +221,14 @@ private[parquet] object ParquetTypesConverter { new ParquetPrimitiveType(repetition, primitiveType.get, name) } else { ctype match { - case ArrayType(elementType: DataType) => { - elementType match { - case StructType(fields) => { // first case: array of structs - val parquetFieldTypes = fields.map( - f => fromDataType(f.dataType, f.name, f.nullable, inArray = false)) - assert( - fields.size > 1, - "Found struct inside array with a single field.. error parsing Catalyst schema") - new ParquetGroupType( - Repetition.REPEATED, - name, - ParquetOriginalType.LIST, - parquetFieldTypes) - } - case _ => { // second case: array of primitive types - val parquetElementType = fromDataType( - elementType, - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - nullable = false, - inArray = true) - ConversionPatterns.listType(repetition, name, parquetElementType) - } - } + case ArrayType(elementType) => { + val parquetElementType = fromDataType( + elementType, + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + nullable = false, + inArray = true) + ConversionPatterns.listType(repetition, name, parquetElementType) } - // TODO: test structs inside arrays case StructType(structFields) => { val fields = structFields.map { field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) @@ -277,11 +236,23 @@ private[parquet] object ParquetTypesConverter { new ParquetGroupType(repetition, name, fields) } case MapType(keyType, valueType) => { + val parquetKeyType = + fromDataType( + keyType, + CatalystConverter.MAP_KEY_SCHEMA_NAME, + false, + inArray = false) + val parquetValueType = + fromDataType( + valueType, + CatalystConverter.MAP_VALUE_SCHEMA_NAME, + true, + inArray = false) ConversionPatterns.mapType( repetition, name, - fromDataType(keyType, "key", false, inArray = false), - fromDataType(valueType, "value", true, inArray = false)) + parquetKeyType, + parquetValueType) } case _ => sys.error(s"Unsupported datatype $ctype") } From 1dc5ac987633421f083240880c2978a1d7ddf9d3 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 12 Apr 2014 20:15:40 +0300 Subject: [PATCH 19/48] First version of WriteSupport for nested types --- .../sql/parquet/ParquetTableOperations.scala | 12 +- .../sql/parquet/ParquetTableSupport.scala | 131 +++++++++++++++++- .../spark/sql/parquet/ParquetTypes.scala | 20 --- 3 files changed, 135 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 65ba1246fbf9a..3e14ec31fd5f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -166,9 +166,15 @@ case class InsertIntoParquetTable( val job = new Job(sc.hadoopConfiguration) - ParquetOutputFormat.setWriteSupportClass( - job, - classOf[org.apache.spark.sql.parquet.RowWriteSupport]) + val writeSupport = + if (child.output.map(_.dataType).forall(_.isPrimitive())) { + logger.info("Initializing MutableRowWriteSupport") + classOf[org.apache.spark.sql.parquet.MutableRowWriteSupport] + } else { + classOf[org.apache.spark.sql.parquet.RowWriteSupport] + } + + ParquetOutputFormat.setWriteSupportClass(job, writeSupport) // TODO: move that to function in object val conf = ContextUtil.getConfiguration(job) 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 ede8fb6204664..e29d416d5cfb7 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 @@ -28,6 +28,10 @@ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.ArrayType +import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.catalyst.types.MapType /** * A `parquet.io.api.RecordMaterializer` for Rows. @@ -97,9 +101,9 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { MessageTypeParser.parseMessageType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) } - private var schema: MessageType = null - private var writer: RecordConsumer = null - private var attributes: Seq[Attribute] = null + private[parquet] var schema: MessageType = null + private[parquet] var writer: RecordConsumer = null + private[parquet] var attributes: Seq[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { schema = if (schema == null) getSchema(configuration) else schema @@ -116,7 +120,6 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { log.debug(s"preparing for write with schema $schema") } - // TODO: add groups (nested fields) override def write(record: Row): Unit = { if (attributes.size > record.size) { throw new IndexOutOfBoundsException( @@ -129,13 +132,131 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { // null values indicate optional fields but we do not check currently if (record(index) != null && record(index) != Nil) { writer.startField(attributes(index).name, index) - ParquetTypesConverter.consumeType(writer, attributes(index).dataType, record, index) + writeValue(attributes(index).dataType, record(index)) writer.endField(attributes(index).name, index) } index = index + 1 } writer.endMessage() } + + private[parquet] def writeValue(schema: DataType, value: Any): Unit = { + schema match { + case t @ ArrayType(_) => writeArray(t, value.asInstanceOf[Row]) + case t @ MapType(_, _) => writeMap(t, value.asInstanceOf[Map[Any, Any]]) + case t @ StructType(_) => writeStruct(t, value.asInstanceOf[Row]) + case _ => writePrimitive(schema.asInstanceOf[PrimitiveType], value) + } + } + + private[parquet] def writePrimitive(schema: PrimitiveType, value: Any): Unit = { + schema match { + case StringType => writer.addBinary( + Binary.fromByteArray( + value.asInstanceOf[String].getBytes("utf-8") + ) + ) + case IntegerType => writer.addInteger(value.asInstanceOf[Int]) + case LongType => writer.addLong(value.asInstanceOf[Long]) + case DoubleType => writer.addDouble(value.asInstanceOf[Double]) + case FloatType => writer.addFloat(value.asInstanceOf[Float]) + case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) + case _ => sys.error(s"Do not know how to writer $schema to consumer") + } + } + + private[parquet] def writeStruct(schema: StructType, struct: Row): Unit = { + val fields = schema.fields.toArray + writer.startGroup() + var i = 0 + while(i < fields.size) { + writer.startField(fields(i).name, i) + writeValue(fields(i).dataType, struct(i)) + writer.endField(fields(i).name, i) + i = i + 1 + } + writer.endGroup() + } + + private[parquet] def writeArray(schema: ArrayType, array: Row): Unit = { + val elementType = schema.elementType + writer.startGroup() + if (array.size > 0) { + writer.startField("values", 0) + writer.startGroup() + var i = 0 + while(i < array.size) { + writeValue(elementType, array(i)) + i = i + 1 + } + writer.endGroup() + writer.endField("values", 0) + } + writer.endGroup() + } + + private[parquet] def writeMap(schema: MapType, map: Map[_, _]): Unit = { + writer.startGroup() + if (map.size > 0) { + writer.startField("map", 0) + writer.startGroup() + writer.startField("key", 0) + for(key <- map.keys) { + writeValue(schema.keyType, key) + } + writer.endField("key", 0) + writer.startField("value", 1) + for(value <- map.values) { + writeValue(schema.valueType, value) + } + writer.endField("value", 1) + writer.endGroup() + writer.endField("map", 0) + } + writer.endGroup() + } +} + +// Optimized for non-nested rows +private[parquet] class MutableRowWriteSupport extends RowWriteSupport { + override def write(record: Row): Unit = { + if (attributes.size > record.size) { + throw new IndexOutOfBoundsException( + s"Trying to write more fields than contained in row (${attributes.size}>${record.size})") + } + + var index = 0 + writer.startMessage() + while(index < attributes.size) { + // null values indicate optional fields but we do not check currently + if (record(index) != null && record(index) != Nil) { + writer.startField(attributes(index).name, index) + consumeType(attributes(index).dataType, record, index) + writer.endField(attributes(index).name, index) + } + index = index + 1 + } + writer.endMessage() + } + + private def consumeType( + ctype: DataType, + record: Row, + index: Int): Unit = { + ctype match { + case StringType => writer.addBinary( + Binary.fromByteArray( + record(index).asInstanceOf[String].getBytes("utf-8") + ) + ) + case IntegerType => writer.addInteger(record.getInt(index)) + case LongType => writer.addLong(record.getLong(index)) + case DoubleType => writer.addDouble(record.getDouble(index)) + case FloatType => writer.addFloat(record.getFloat(index)) + case BooleanType => writer.addBoolean(record.getBoolean(index)) + case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer") + } + } } private[parquet] object RowWriteSupport { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 867dc505e095a..50f06d5467505 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -259,26 +259,6 @@ private[parquet] object ParquetTypesConverter { } } - def consumeType( - consumer: RecordConsumer, - ctype: DataType, - record: Row, - index: Int): Unit = { - ctype match { - case StringType => consumer.addBinary( - Binary.fromByteArray( - record(index).asInstanceOf[String].getBytes("utf-8") - ) - ) - case IntegerType => consumer.addInteger(record.getInt(index)) - case LongType => consumer.addLong(record.getLong(index)) - case DoubleType => consumer.addDouble(record.getDouble(index)) - case FloatType => consumer.addFloat(record.getFloat(index)) - case BooleanType => consumer.addBoolean(record.getBoolean(index)) - case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer") - } - } - def getSchema(schemaString: String) : MessageType = MessageTypeParser.parseMessageType(schemaString) From e99cc514397f217f307807e1d6e33e1aea9ada84 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 13 Apr 2014 11:23:38 +0300 Subject: [PATCH 20/48] Fixing nested WriteSupport and adding tests --- .../sql/parquet/ParquetTableSupport.scala | 61 +++++++++++-------- .../spark/sql/parquet/ParquetTestData.scala | 4 +- .../spark/sql/parquet/ParquetTypes.scala | 23 ++++--- .../spark/sql/parquet/ParquetQuerySuite.scala | 49 +++++++++++++++ 4 files changed, 99 insertions(+), 38 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index e29d416d5cfb7..567574516883d 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 @@ -141,41 +141,49 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } private[parquet] def writeValue(schema: DataType, value: Any): Unit = { - schema match { - case t @ ArrayType(_) => writeArray(t, value.asInstanceOf[Row]) - case t @ MapType(_, _) => writeMap(t, value.asInstanceOf[Map[Any, Any]]) - case t @ StructType(_) => writeStruct(t, value.asInstanceOf[Row]) - case _ => writePrimitive(schema.asInstanceOf[PrimitiveType], value) + if (value != null && value != Nil) { + schema match { + case t @ ArrayType(_) => writeArray(t, value.asInstanceOf[Row]) + case t @ MapType(_, _) => writeMap(t, value.asInstanceOf[Map[Any, Any]]) + case t @ StructType(_) => writeStruct(t, value.asInstanceOf[Row]) + case _ => writePrimitive(schema.asInstanceOf[PrimitiveType], value) + } } } private[parquet] def writePrimitive(schema: PrimitiveType, value: Any): Unit = { - schema match { - case StringType => writer.addBinary( - Binary.fromByteArray( - value.asInstanceOf[String].getBytes("utf-8") + if (value != null && value != Nil) { + schema match { + case StringType => writer.addBinary( + Binary.fromByteArray( + value.asInstanceOf[String].getBytes("utf-8") + ) ) - ) - case IntegerType => writer.addInteger(value.asInstanceOf[Int]) - case LongType => writer.addLong(value.asInstanceOf[Long]) - case DoubleType => writer.addDouble(value.asInstanceOf[Double]) - case FloatType => writer.addFloat(value.asInstanceOf[Float]) - case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) - case _ => sys.error(s"Do not know how to writer $schema to consumer") + case IntegerType => writer.addInteger(value.asInstanceOf[Int]) + case LongType => writer.addLong(value.asInstanceOf[Long]) + case DoubleType => writer.addDouble(value.asInstanceOf[Double]) + case FloatType => writer.addFloat(value.asInstanceOf[Float]) + case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) + case _ => sys.error(s"Do not know how to writer $schema to consumer") + } } } private[parquet] def writeStruct(schema: StructType, struct: Row): Unit = { - val fields = schema.fields.toArray - writer.startGroup() - var i = 0 - while(i < fields.size) { - writer.startField(fields(i).name, i) - writeValue(fields(i).dataType, struct(i)) - writer.endField(fields(i).name, i) - i = i + 1 + if (struct != null && struct != Nil) { + val fields = schema.fields.toArray + writer.startGroup() + var i = 0 + while(i < fields.size) { + if (struct(i) != null && struct(i) != Nil) { + writer.startField(fields(i).name, i) + writeValue(fields(i).dataType, struct(i)) + writer.endField(fields(i).name, i) + } + i = i + 1 + } + writer.endGroup() } - writer.endGroup() } private[parquet] def writeArray(schema: ArrayType, array: Row): Unit = { @@ -183,18 +191,17 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { writer.startGroup() if (array.size > 0) { writer.startField("values", 0) - writer.startGroup() var i = 0 while(i < array.size) { writeValue(elementType, array(i)) i = i + 1 } - writer.endGroup() writer.endField("values", 0) } writer.endGroup() } + // TODO: this does not allow null values! Should these be supported? private[parquet] def writeMap(schema: MapType, map: Map[_, _]): Unit = { writer.startGroup() if (map.size > 0) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 8a34ee3b92eff..c33d04925653f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -182,13 +182,13 @@ private[sql] object ParquetTestData { |optional group data1 { |repeated group map { |required binary key; - |optional int32 value; + |required int32 value; |} |} |required group data2 { |repeated group map { |required binary key; - |optional group value { + |required group value { |required int64 payload1; |optional binary payload2; |} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 50f06d5467505..bee469a5ddd19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -64,13 +64,14 @@ private[parquet] object ParquetTypesConverter { *
      *
    • Primitive types are converter to the corresponding primitive type.
    • *
    • Group types that have a single field that is itself a group, which has repetition - * level `REPEATED` are treated as follows:
        - *
      • If the nested group has name `values` and repetition level `REPEATED`, the - * surrounding group is converted into an [[ArrayType]] with the - * corresponding field type (primitive or complex) as element type.
      • - *
      • If the nested group has name `map`, repetition level `REPEATED` and two fields - * (named `key` and `value`), the surrounding group is converted into a [[MapType]] - * with the corresponding key and value (value possibly complex) types.
      • + * level `REPEATED`, are treated as follows:
          + *
        • If the nested group has name `values`, the surrounding group is converted + * into an [[ArrayType]] with the corresponding field type (primitive or + * complex) as element type.
        • + *
        • If the nested group has name `map` and two fields (named `key` and `value`), + * the surrounding group is converted into a [[MapType]] + * with the corresponding key and value (value possibly complex) types. + * Note that we currently assume map values are not nullable.
        • *
        • Other group types are converted into a [[StructType]] with the corresponding * field types.
        *
      @@ -121,7 +122,9 @@ private[parquet] object ParquetTypesConverter { keyValueGroup.getFieldCount == 2, "Parquet Map type malformatted: nested group should have 2 (key, value) fields!") val keyType = toDataType(keyValueGroup.getFields.apply(0)) + assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) val valueType = toDataType(keyValueGroup.getFields.apply(1)) + assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) new MapType(keyType, valueType) } case _ => { @@ -129,7 +132,9 @@ private[parquet] object ParquetTypesConverter { if (correspondsToMap(groupType)) { // MapType val keyValueGroup = groupType.getFields.apply(0).asGroupType() val keyType = toDataType(keyValueGroup.getFields.apply(0)) + assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) val valueType = toDataType(keyValueGroup.getFields.apply(1)) + assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) new MapType(keyType, valueType) } else if (correspondsToArray(groupType)) { // ArrayType val elementType = toDataType(groupType.getFields.apply(0)) @@ -240,13 +245,13 @@ private[parquet] object ParquetTypesConverter { fromDataType( keyType, CatalystConverter.MAP_KEY_SCHEMA_NAME, - false, + nullable = false, inArray = false) val parquetValueType = fromDataType( valueType, CatalystConverter.MAP_VALUE_SCHEMA_NAME, - true, + nullable = false, inArray = false) ConversionPatterns.mapType( repetition, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index ea940184ca4e4..85efc69ea9a8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -553,6 +553,55 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result2(0)(1) === "the answer") } + test("Writing out Addressbook and reading it back in") { + implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] + val tmpdir = Utils.createTempDir() + val result = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir1.toString) + .toSchemaRDD + result.saveAsParquetFile(tmpdir.toString) + TestSQLContext + .parquetFile(tmpdir.toString) + .toSchemaRDD + .registerAsTable("tmpcopy") + val tmpdata = sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() + assert(tmpdata.size === 2) + assert(tmpdata(0).size === 2) + assert(tmpdata(0)(0) === "Julien Le Dem") + assert(tmpdata(0)(1) === "Chris Aniszczyk") + assert(tmpdata(1)(0) === "A. Nonymous") + assert(tmpdata(1)(1) === null) + Utils.deleteRecursively(tmpdir) + } + + test("Writing out Map and reading it back in") { + implicit def anyToMap(value: Any) = value.asInstanceOf[Map[String, Row]] + val data = TestSQLContext + .parquetFile(ParquetTestData.testNestedDir4.toString) + .toSchemaRDD + val tmpdir = Utils.createTempDir() + data.saveAsParquetFile(tmpdir.toString) + TestSQLContext + .parquetFile(tmpdir.toString) + .toSchemaRDD + .registerAsTable("tmpmapcopy") + val result1 = sql("SELECT data2 FROM tmpmapcopy").collect() + assert(result1.size === 1) + val entry1 = result1(0)(0).getOrElse("7", null) + assert(entry1 != null) + assert(entry1(0) === 42) + assert(entry1(1) === "the answer") + val entry2 = result1(0)(0).getOrElse("8", null) + assert(entry2 != null) + assert(entry2(0) === 49) + assert(entry2(1) === null) + val result2 = sql("SELECT data2[7].payload1, data2[7].payload2 FROM tmpmapcopy").collect() + assert(result2.size === 1) + assert(result2(0)(0) === 42.toLong) + assert(result2(0)(1) === "the answer") + Utils.deleteRecursively(tmpdir) + } + /** * Creates an empty SchemaRDD backed by a ParquetRelation. * From adc1258f6ebc2c85d38cceb2dbaf45bb53fad86f Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 13 Apr 2014 11:47:58 +0300 Subject: [PATCH 21/48] Optimizing imports --- .../spark/sql/catalyst/plans/logical/LogicalPlan.scala | 10 +--------- .../apache/spark/sql/parquet/ParquetTableSupport.scala | 3 --- .../org/apache/spark/sql/parquet/ParquetTypes.scala | 6 +++--- 3 files changed, 4 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 906339c73d95a..7ff2287fbee03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -20,16 +20,8 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees -import scala.util.matching.Regex -import org.apache.spark.sql.catalyst.types.ArrayType -import org.apache.spark.sql.catalyst.expressions.GetField -import org.apache.spark.sql.catalyst.types.StructType -import org.apache.spark.sql.catalyst.types.MapType -import scala.Some -import org.apache.spark.sql.catalyst.expressions.Alias -import org.apache.spark.sql.catalyst.expressions.GetItem +import org.apache.spark.sql.catalyst.types._ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => 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 567574516883d..47066ba540612 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 @@ -29,9 +29,6 @@ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.ArrayType -import org.apache.spark.sql.catalyst.types.StructType -import org.apache.spark.sql.catalyst.types.MapType /** * A `parquet.io.api.RecordMaterializer` for Rows. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index bee469a5ddd19..c2e1d918f96e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -26,12 +26,12 @@ import org.apache.hadoop.mapreduce.Job import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter} import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData} import parquet.hadoop.util.ContextUtil -import parquet.io.api.{Binary, RecordConsumer} -import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser, GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns} +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser} +import parquet.schema.{GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import parquet.schema.Type.Repetition -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute, Row} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} import org.apache.spark.sql.catalyst.types._ // Implicits From f466ff02929e8c7057710c8b4deeeffab1166414 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 13 Apr 2014 20:06:45 +0300 Subject: [PATCH 22/48] Added ParquetAvro tests and revised Array conversion --- project/SparkBuild.scala | 3 + .../spark/sql/parquet/ParquetConverter.scala | 6 +- .../spark/sql/parquet/ParquetTestData.scala | 28 ++-- .../spark/sql/parquet/ParquetQuerySuite.scala | 158 ++++++++++++++++-- 4 files changed, 169 insertions(+), 26 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7bb39dc77120b..605fd4839fa7d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -503,6 +503,9 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.twitter" % "parquet-column" % parquetVersion, "com.twitter" % "parquet-hadoop" % parquetVersion, + "com.twitter" % "parquet-avro" % parquetVersion % "test", + // here we need version >= 1.7.5 because of AVRO-1274 + "org.apache.avro" % "avro" % "1.7.6" % "test" "com.fasterxml.jackson.core" % "jackson-databind" % "2.3.0" // json4s-jackson 3.2.6 requires jackson-databind 2.3.0. ), initialCommands in console := 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 3cb25925c7b08..281357f22914f 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 @@ -32,8 +32,10 @@ private[parquet] object CatalystConverter { // The type internally used for fields type FieldType = StructField - // This is mostly Parquet convention (see, e.g., `ConversionPatterns`) - val ARRAY_ELEMENTS_SCHEMA_NAME = "values" + // 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_ELEMENTS_SCHEMA_NAME = "array" val MAP_KEY_SCHEMA_NAME = "key" val MAP_VALUE_SCHEMA_NAME = "value" val MAP_SCHEMA_NAME = "map" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index c33d04925653f..d3449a6cbf77e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -121,10 +121,10 @@ private[sql] object ParquetTestData { |message AddressBook { |required binary owner; |optional group ownerPhoneNumbers { - |repeated binary values; + |repeated binary array; |} |optional group contacts { - |repeated group values { + |repeated group array { |required binary name; |optional binary phoneNumber; |} @@ -139,18 +139,18 @@ private[sql] object ParquetTestData { |required int32 firstInt; |optional int32 secondInt; |optional group longs { - |repeated int64 values; + |repeated int64 array; |} |required group entries { - |repeated group values { + |repeated group array { |required double value; |optional boolean truth; |} |} |optional group outerouter { - |repeated group values { - |repeated group values { - |repeated int32 values; + |repeated group array { + |repeated group array { + |repeated int32 array; |} |} |} @@ -162,10 +162,10 @@ private[sql] object ParquetTestData { |message TestNested3 { |required int32 x; |optional group booleanNumberPairs { - |repeated group values { + |repeated group array { |required int32 key; |optional group value { - |repeated group values { + |repeated group array { |required double nestedValue; |optional boolean truth; |} @@ -273,9 +273,9 @@ private[sql] object ParquetTestData { val r1 = new SimpleGroup(schema) r1.add(0, "Julien Le Dem") r1.addGroup(1) - .append("values", "555 123 4567") - .append("values", "555 666 1337") - .append("values", "XXX XXX XXXX") + .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "555 123 4567") + .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "555 666 1337") + .append(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, "XXX XXX XXXX") val contacts = r1.addGroup(2) contacts.addGroup(0) .append("name", "Dmitriy Ryaboy") @@ -398,10 +398,10 @@ private[sql] object ParquetTestData { val fs: FileSystem = path.getFileSystem(configuration) val schema: MessageType = MessageTypeParser.parseMessageType(schemaString) assert(schema != null) - val outputStatus: FileStatus = fs.getFileStatus(path) + val outputStatus: FileStatus = fs.getFileStatus(new Path(path.toString)) val footers = ParquetFileReader.readFooter(configuration, outputStatus) assert(footers != null) - val reader = new ParquetReader(path, new GroupReadSupport()) + val reader = new ParquetReader(new Path(path.toString), new GroupReadSupport()) val first = reader.read() assert(first != null) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 85efc69ea9a8b..6e77fdb3e70da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -19,9 +19,13 @@ package org.apache.spark.sql.parquet import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} +import org.apache.avro.{SchemaBuilder, Schema} +import org.apache.avro.generic.{GenericData, GenericRecord} + import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.mapreduce.Job +import parquet.avro.AvroParquetWriter import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil import parquet.schema.MessageTypeParser @@ -34,11 +38,12 @@ import org.apache.spark.sql.SchemaRDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.util.Utils +import org.apache.spark.sql.SchemaRDD +import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, DataType} -import org.apache.spark.sql.{parquet, SchemaRDD} import org.apache.spark.sql.catalyst.expressions.AttributeReference -import scala.Tuple2 -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.util.Utils // Implicits import org.apache.spark.sql.test.TestSQLContext._ @@ -398,9 +403,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Importing nested Parquet file (Addressbook)") { implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] - ParquetTestData.readNestedFile( - ParquetTestData.testNestedDir1, - ParquetTestData.testNestedSchema1) val result = TestSQLContext .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD @@ -426,9 +428,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Importing nested Parquet file (nested numbers)") { implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] - ParquetTestData.readNestedFile( - ParquetTestData.testNestedDir2, - ParquetTestData.testNestedSchema2) val result = TestSQLContext .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD @@ -602,6 +601,145 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(tmpdir) } + test("Importing data generated with Avro") { + val tmpdir = Utils.createTempDir() + val file: File = new File(tmpdir, "test.avro") + + val primitiveArrayType: Schema = SchemaBuilder.array.items.intType + val complexArrayType: Schema = SchemaBuilder.array.items.map.values.stringType + val primitiveMapType: Schema = SchemaBuilder.map.values.booleanType + val complexMapType: Schema = SchemaBuilder.map.values.array.items.floatType + val schema: Schema = SchemaBuilder + .record("TestRecord") + .namespace("") + .fields + .name("testInt") + .`type`. + intType + .noDefault + .name("testDouble") + .`type` + .doubleType + .noDefault + .name("testString") + .`type` + .nullable + .stringType + .stringDefault("") + .name("testPrimitiveArray") + .`type`(primitiveArrayType) + .noDefault + .name("testComplexArray") + .`type`(complexArrayType) + .noDefault + .name("testPrimitiveMap") + .`type`(primitiveMapType) + .noDefault + .name("testComplexMap") + .`type`(complexMapType) + .noDefault + .endRecord + + val record1: GenericRecord = new GenericData.Record(schema) + + // primitive fields + record1.put("testInt", 256) + record1.put("testDouble", 0.5) + record1.put("testString", "foo") + + val primitiveArrayData = new GenericData.Array[Integer](10, primitiveArrayType) + val complexArrayData: GenericData.Array[java.util.Map[String, String]] = + new GenericData.Array[java.util.Map[String, String]](10, SchemaBuilder.array.items.map.values.stringType) + + // two arrays: one primitive (array of ints), one complex (array of string->string maps) + primitiveArrayData.add(1) + primitiveArrayData.add(2) + primitiveArrayData.add(3) + val map1 = new java.util.HashMap[String, String] + map1.put("key11", "data11") + map1.put("key12", "data12") + val map2 = new java.util.HashMap[String, String] + map2.put("key21", "data21") + map2.put("key22", "data22") + complexArrayData.add(0, map1) + complexArrayData.add(1, map2) + + record1.put("testPrimitiveArray", primitiveArrayData) + record1.put("testComplexArray", complexArrayData) + + // two maps: one primitive (string->boolean), one complex (string->array of floats) + val primitiveMap = new java.util.HashMap[String, Boolean](10) + primitiveMap.put("key1", true) + primitiveMap.put("key2", false) + val complexMap = new java.util.HashMap[String, GenericData.Array[Float]](10) + val value1: GenericData.Array[Float] = new GenericData.Array[Float](10, SchemaBuilder.array.items.floatType) + value1.add(0.1f) + value1.add(0.2f) + value1.add(0.3f) + complexMap.put("compKey1", value1) + val value2: GenericData.Array[Float] = new GenericData.Array[Float](10, SchemaBuilder.array.items.floatType) + value2.add(1.1f) + value2.add(1.2f) + value2.add(1.3f) + complexMap.put("compKey2", value2) + + record1.put("testPrimitiveMap", primitiveMap) + record1.put("testComplexMap", complexMap) + + // TODO: test array or map with value type Avro record + + val writer = new AvroParquetWriter[GenericRecord](new Path(file.toString), schema) + writer.write(record1) + writer.close() + + val data = TestSQLContext + .parquetFile(tmpdir.toString) + .toSchemaRDD + data.registerAsTable("avroTable") + val resultPrimitives = sql("SELECT testInt, testDouble, testString FROM avroTable").collect() + assert(resultPrimitives(0)(0) === 256) + assert(resultPrimitives(0)(1) === 0.5) + assert(resultPrimitives(0)(2) === "foo") + val resultPrimitiveArray = sql("SELECT testPrimitiveArray FROM avroTable").collect() + assert(resultPrimitiveArray(0)(0).asInstanceOf[Row](0) === 1) + assert(resultPrimitiveArray(0)(0).asInstanceOf[Row](1) === 2) + assert(resultPrimitiveArray(0)(0).asInstanceOf[Row](2) === 3) + val resultComplexArray = sql("SELECT testComplexArray FROM avroTable").collect() + assert(resultComplexArray(0)(0).asInstanceOf[Row].size === 2) + assert( + resultComplexArray(0)(0) + .asInstanceOf[Row] + .apply(0) + .asInstanceOf[Map[String, String]].get("key11").get.equals("data11")) + assert( + resultComplexArray(0)(0) + .asInstanceOf[Row] + .apply(1) + .asInstanceOf[Map[String, String]].get("key22").get.equals("data22")) + val resultPrimitiveMap = sql("SELECT testPrimitiveMap FROM avroTable").collect() + assert( + resultPrimitiveMap(0)(0) + .asInstanceOf[Map[String, Boolean]].get("key1").get === true) + assert( + resultPrimitiveMap(0)(0) + .asInstanceOf[Map[String, Boolean]].get("key2").get === false) + val resultComplexMap = sql("SELECT testComplexMap FROM avroTable").collect() + val mapResult1 = + resultComplexMap(0)(0) + .asInstanceOf[Map[String, Row]] + .get("compKey1") + .get + val mapResult2 = + resultComplexMap(0)(0) + .asInstanceOf[Map[String, Row]] + .get("compKey2") + .get + assert(mapResult1(0) === 0.1f) + assert(mapResult1(2) === 0.3f) + assert(mapResult2(0) === 1.1f) + assert(mapResult2(2) === 1.3f) + } + /** * Creates an empty SchemaRDD backed by a ParquetRelation. * @@ -613,6 +751,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val attributes = schema.map(t => new AttributeReference(t._1, t._2)()) new SchemaRDD( TestSQLContext, - parquet.ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration)) + ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration)) } } From 79d81d5db3f73c4432f678e032cd2378d594f20a Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 13 Apr 2014 20:23:12 +0300 Subject: [PATCH 23/48] Replacing field names for array and map in WriteSupport --- .../spark/sql/parquet/ParquetTableSupport.scala | 16 ++++++++-------- .../spark/sql/parquet/ParquetQuerySuite.scala | 2 ++ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 47066ba540612..b50c51c0a8b2f 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 @@ -187,13 +187,13 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { val elementType = schema.elementType writer.startGroup() if (array.size > 0) { - writer.startField("values", 0) + writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) var i = 0 while(i < array.size) { writeValue(elementType, array(i)) i = i + 1 } - writer.endField("values", 0) + writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) } writer.endGroup() } @@ -202,20 +202,20 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] def writeMap(schema: MapType, map: Map[_, _]): Unit = { writer.startGroup() if (map.size > 0) { - writer.startField("map", 0) + writer.startField(CatalystConverter.MAP_SCHEMA_NAME, 0) writer.startGroup() - writer.startField("key", 0) + writer.startField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) for(key <- map.keys) { writeValue(schema.keyType, key) } - writer.endField("key", 0) - writer.startField("value", 1) + writer.endField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) + writer.startField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) for(value <- map.values) { writeValue(schema.valueType, value) } - writer.endField("value", 1) + writer.endField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) writer.endGroup() - writer.endField("map", 0) + writer.endField(CatalystConverter.MAP_SCHEMA_NAME, 0) } writer.endGroup() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 6e77fdb3e70da..9a65e785f5ab3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -553,6 +553,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Writing out Addressbook and reading it back in") { + // TODO: find out why CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME + // has no effect in this test case implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val tmpdir = Utils.createTempDir() val result = TestSQLContext From 619c397f46cb9b9496e0133f02f1b634c58b703f Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Mon, 14 Apr 2014 16:01:47 +0300 Subject: [PATCH 24/48] Completing Map testcase --- .../spark/sql/parquet/ParquetQuerySuite.scala | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 9a65e785f5ab3..588b5fe468881 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -586,20 +586,23 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .parquetFile(tmpdir.toString) .toSchemaRDD .registerAsTable("tmpmapcopy") - val result1 = sql("SELECT data2 FROM tmpmapcopy").collect() + val result1 = sql("SELECT data1[key2] FROM tmpmapcopy").collect() assert(result1.size === 1) - val entry1 = result1(0)(0).getOrElse("7", null) + assert(result1(0)(0) === 2) + val result2 = sql("SELECT data2 FROM tmpmapcopy").collect() + assert(result2.size === 1) + val entry1 = result2(0)(0).getOrElse("7", null) assert(entry1 != null) assert(entry1(0) === 42) assert(entry1(1) === "the answer") - val entry2 = result1(0)(0).getOrElse("8", null) + val entry2 = result2(0)(0).getOrElse("8", null) assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result2 = sql("SELECT data2[7].payload1, data2[7].payload2 FROM tmpmapcopy").collect() - assert(result2.size === 1) - assert(result2(0)(0) === 42.toLong) - assert(result2(0)(1) === "the answer") + val result3 = sql("SELECT data2[7].payload1, data2[7].payload2 FROM tmpmapcopy").collect() + assert(result3.size === 1) + assert(result3(0)(0) === 42.toLong) + assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } From c52ff2c9afb411dbde3219fc5115567b31a20a28 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 19 Apr 2014 17:30:35 +0300 Subject: [PATCH 25/48] Adding native-array converter --- .../spark/sql/parquet/ParquetConverter.scala | 139 +++++++++++++++++- 1 file changed, 132 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 281357f22914f..e8ebfdecbe1d3 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,16 +17,17 @@ package org.apache.spark.sql.parquet -import scala.collection.mutable.{Buffer, ArrayBuffer} +import scala.collection.mutable.{Buffer, ArrayBuffer, HashMap} +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.runtimeMirror import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType -import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} import org.apache.spark.sql.parquet.CatalystConverter.FieldType -import scala.collection.mutable +import org.apache.spark.util.Utils private[parquet] object CatalystConverter { // The type internally used for fields @@ -46,6 +47,11 @@ private[parquet] object CatalystConverter { parent: CatalystConverter): Converter = { val fieldType: DataType = field.dataType fieldType match { + // For native JVM types we use a converter with native arrays + case ArrayType(elementType: NativeType) => { + new CatalystNativeArrayConverter(elementType, fieldIndex, parent) + } + // This is for other types of arrays, including those with nested fields case ArrayType(elementType: DataType) => { new CatalystArrayConverter(elementType, fieldIndex, parent) } @@ -322,8 +328,17 @@ object CatalystArrayConverter { val INITIAL_ARRAY_SIZE = 20 } -// this is for single-element groups of primitive or complex types -// Note: AvroParquet only uses arrays for primitive types (?) +/** + * 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.catalyst.types.ArrayType]]. + * + * @param elementType The type of the array elements + * @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, @@ -353,7 +368,8 @@ private[parquet] class CatalystArrayConverter( // arrays have only one (repeated) field, which is its elements override val size = 1 - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit ={ + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + // fieldIndex is ignored (assumed to be zero but not checked) buffer += value } @@ -380,6 +396,115 @@ private[parquet] class CatalystArrayConverter( override def getCurrentRecord: Row = throw new UnsupportedOperationException } +private[parquet] class CatalystNativeArrayConverter[T <: NativeType]( + val elementType: NativeType, + val index: Int, + protected[parquet] val parent: CatalystConverter, + protected[parquet] var capacity: Int = CatalystArrayConverter.INITIAL_ARRAY_SIZE) + extends GroupConverter with CatalystConverter { + + // similar comment as in [[Decoder]]: this should probably be in NativeType + private val classTag = { + val mirror = runtimeMirror(Utils.getSparkClassLoader) + ClassTag[T#JvmType](mirror.runtimeClass(elementType.tag.tpe)) + } + + private var buffer: Array[T#JvmType] = 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 + + // Overriden here to avoid auto-boxing for primitive types + override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { + checkGrowBuffer() + buffer(elements) = value.asInstanceOf[T#JvmType] + elements += 1 + } + + override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { + checkGrowBuffer() + buffer(elements) = value.asInstanceOf[T#JvmType] + elements += 1 + } + + override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { + checkGrowBuffer() + buffer(elements) = value.asInstanceOf[T#JvmType] + elements += 1 + } + + override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { + checkGrowBuffer() + buffer(elements) = value.asInstanceOf[T#JvmType] + elements += 1 + } + + override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { + checkGrowBuffer() + buffer(elements) = value.asInstanceOf[T#JvmType] + elements += 1 + } + + override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { + checkGrowBuffer() + buffer(elements) = value.getBytes.asInstanceOf[T#JvmType] + elements += 1 + } + + override protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit = { + checkGrowBuffer() + buffer(elements) = value.toStringUsingUTF8.asInstanceOf[T#JvmType] + elements += 1 + } + + override protected[parquet] def clearBuffer(): Unit = { + elements = 0 + } + + override def start(): Unit = {} + + override def end(): Unit = { + assert(parent != null) + parent.updateField( + index, + new GenericRow { + // TODO: it would be much nicer to use a view here but GenericRow requires an Array + // TODO: we should avoid using GenericRow as a wrapper but [[GetField]] current + // requires that + override val values = buffer.slice(0, elements).map(_.asInstanceOf[Any]) + }) + clearBuffer() + } + + // Should be only called in root group converter! + override def getCurrentRecord: Row = throw new UnsupportedOperationException + + private def checkGrowBuffer(): Unit = { + if (elements >= capacity) { + val newCapacity = 2 * capacity + val tmp: Array[T#JvmType] = classTag.newArray(newCapacity) + Array.copy(buffer, 0, tmp, 0, capacity) + buffer = tmp + capacity = newCapacity + } + } +} + // this is for multi-element groups of primitive or complex types // that have repetition level optional or required (so struct fields) private[parquet] class CatalystStructConverter( @@ -407,7 +532,7 @@ private[parquet] class CatalystMapConverter( override protected[parquet] val parent: CatalystConverter) extends GroupConverter with CatalystConverter { - private val map = new mutable.HashMap[Any, Any]() + private val map = new HashMap[Any, Any]() private val keyValueConverter = new GroupConverter with CatalystConverter { private var currentKey: Any = null From 431f00ff3ecaa119ce90a2331ff5dd0c07ab897f Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 19 Apr 2014 18:37:15 +0300 Subject: [PATCH 26/48] Fixing problems introduced during rebase --- .../spark/sql/parquet/ParquetQuerySuite.scala | 27 +++++-------------- 1 file changed, 6 insertions(+), 21 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 588b5fe468881..a42d2cbb1f855 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.parquet +import java.io.File + import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} import org.apache.avro.{SchemaBuilder, Schema} @@ -31,18 +33,13 @@ import parquet.hadoop.util.ContextUtil import parquet.schema.MessageTypeParser import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.util.getTempFilePath -import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.TestData -import org.apache.spark.sql.SchemaRDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.IntegerType -import org.apache.spark.util.Utils +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.TestData import org.apache.spark.sql.SchemaRDD import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, DataType} -import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.util.Utils // Implicits @@ -557,6 +554,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA // has no effect in this test case implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val tmpdir = Utils.createTempDir() + Utils.deleteRecursively(tmpdir) val result = TestSQLContext .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD @@ -581,6 +579,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD val tmpdir = Utils.createTempDir() + Utils.deleteRecursively(tmpdir) data.saveAsParquetFile(tmpdir.toString) TestSQLContext .parquetFile(tmpdir.toString) @@ -744,18 +743,4 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(mapResult2(0) === 1.1f) assert(mapResult2(2) === 1.3f) } - - /** - * Creates an empty SchemaRDD backed by a ParquetRelation. - * - * TODO: since this is so experimental it is better to have it here and not - * in SQLContext. Also note that when creating new AttributeReferences - * one needs to take care not to create duplicate Attribute ID's. - */ - private def createParquetFile(path: String, schema: (Tuple2[String, DataType])*): SchemaRDD = { - val attributes = schema.map(t => new AttributeReference(t._1, t._2)()) - new SchemaRDD( - TestSQLContext, - ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration)) - } } From a6b4f050c02e18409e052ae9c9e2489deac09b0d Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 20 Apr 2014 11:58:18 +0300 Subject: [PATCH 27/48] Cleaning up ArrayConverter, moving classTag to NativeType, adding NativeRow --- .../spark/sql/catalyst/expressions/Row.scala | 61 +++++++++++++++ .../spark/sql/catalyst/types/dataTypes.scala | 9 ++- .../spark/sql/parquet/ParquetConverter.scala | 74 ++++++++----------- 3 files changed, 101 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 74ae723686cfe..b587610749a15 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -206,6 +206,67 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { override def copy() = new GenericRow(values.clone()) } +// TODO: this is an awful lot of code duplication. If values would be covariant we could reuse +// much of GenericRow +class NativeRow[T](protected[catalyst] val values: Array[T]) extends Row { + + /** No-arg constructor for serialization. */ + def this() = this(null) + + def this(elementType: NativeType, size: Int) = + this(elementType.classTag.newArray(size).asInstanceOf[Array[T]]) + + def iterator = values.iterator + + def length = values.length + + def apply(i: Int) = values(i) + + def isNullAt(i: Int) = values(i) == null + + def getInt(i: Int): Int = { + if (values(i) == null) sys.error("Failed to check null bit for primitive int value.") + values(i).asInstanceOf[Int] + } + + def getLong(i: Int): Long = { + if (values(i) == null) sys.error("Failed to check null bit for primitive long value.") + values(i).asInstanceOf[Long] + } + + def getDouble(i: Int): Double = { + if (values(i) == null) sys.error("Failed to check null bit for primitive double value.") + values(i).asInstanceOf[Double] + } + + def getFloat(i: Int): Float = { + if (values(i) == null) sys.error("Failed to check null bit for primitive float value.") + values(i).asInstanceOf[Float] + } + + def getBoolean(i: Int): Boolean = { + if (values(i) == null) sys.error("Failed to check null bit for primitive boolean value.") + values(i).asInstanceOf[Boolean] + } + + def getShort(i: Int): Short = { + if (values(i) == null) sys.error("Failed to check null bit for primitive short value.") + values(i).asInstanceOf[Short] + } + + def getByte(i: Int): Byte = { + if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") + values(i).asInstanceOf[Byte] + } + + def getString(i: Int): String = { + if (values(i) == null) sys.error("Failed to check null bit for primitive String value.") + values(i).asInstanceOf[String] + } + + def copy() = this +} + class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 74cfa5962fe3b..a3484cc3de870 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -19,9 +19,11 @@ package org.apache.spark.sql.catalyst.types import java.sql.Timestamp -import scala.reflect.runtime.universe.{typeTag, TypeTag} +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.{typeTag, TypeTag, runtimeMirror} import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.util.Utils abstract class DataType { /** Matches any expression that evaluates to this DataType */ @@ -43,6 +45,11 @@ abstract class NativeType extends DataType { type JvmType @transient val tag: TypeTag[JvmType] val ordering: Ordering[JvmType] + + @transient val classTag = { + val mirror = runtimeMirror(Utils.getSparkClassLoader) + ClassTag[JvmType](mirror.runtimeClass(tag.tpe)) + } } case object StringType extends NativeType with PrimitiveType { 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 e8ebfdecbe1d3..0463433a66ecd 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 @@ -18,16 +18,13 @@ package org.apache.spark.sql.parquet import scala.collection.mutable.{Buffer, ArrayBuffer, HashMap} -import scala.reflect.ClassTag -import scala.reflect.runtime.universe.runtimeMirror import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions.{NativeRow, GenericRow, Row, Attribute} import org.apache.spark.sql.parquet.CatalystConverter.FieldType -import org.apache.spark.util.Utils private[parquet] object CatalystConverter { // The type internally used for fields @@ -83,7 +80,7 @@ private[parquet] object CatalystConverter { val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) // For non-nested types we use the optimized Row converter if (attributes.forall(a => ParquetTypesConverter.isPrimitiveType(a.dataType))) { - new MutableRowGroupConverter(attributes) + new PrimitiveRowGroupConverter(attributes) } else { new CatalystGroupConverter(attributes) } @@ -170,6 +167,9 @@ private[parquet] class CatalystGroupConverter( def getCurrentRecord: Row = { 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 GenericRow(current.toArray) } @@ -180,14 +180,9 @@ private[parquet] class CatalystGroupConverter( current.update(fieldIndex, value) } - override protected[parquet] def clearBuffer(): Unit = { - // TODO: reuse buffer? - buffer = new ArrayBuffer[Row](CatalystArrayConverter.INITIAL_ARRAY_SIZE) - } + override protected[parquet] def clearBuffer(): Unit = buffer.clear() override def start(): Unit = { - // TODO: reuse buffer? - // Allocate new array in the root converter (others will be called clearBuffer() on) current = ArrayBuffer.fill(schema.length)(null) converters.foreach { converter => if (!converter.isPrimitive) { @@ -196,12 +191,10 @@ private[parquet] class CatalystGroupConverter( } } - // TODO: think about reusing the buffer override def end(): Unit = { if (!isRootConverter) { assert(current!=null) // there should be no empty groups buffer.append(new GenericRow(current.toArray)) - // TODO: use iterators if possible, avoid Row wrapping parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]])) } } @@ -212,7 +205,7 @@ private[parquet] class CatalystGroupConverter( * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his * converter is optimized for rows of primitive types (non-nested records). */ -private[parquet] class MutableRowGroupConverter( +private[parquet] class PrimitiveRowGroupConverter( protected[parquet] val schema: Seq[FieldType], protected[parquet] var current: ParquetRelation.RowType) extends GroupConverter with CatalystConverter { @@ -334,7 +327,7 @@ object CatalystArrayConverter { * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an * [[org.apache.spark.sql.catalyst.types.ArrayType]]. * - * @param elementType The type of the array elements + * @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 @@ -345,8 +338,6 @@ private[parquet] class CatalystArrayConverter( protected[parquet] val parent: CatalystConverter, protected[parquet] var buffer: Buffer[Any]) extends GroupConverter with CatalystConverter { - // TODO: In the future consider using native arrays instead of buffer for - // primitive types for performance reasons def this(elementType: DataType, index: Int, parent: CatalystConverter) = this( @@ -374,8 +365,7 @@ private[parquet] class CatalystArrayConverter( } override protected[parquet] def clearBuffer(): Unit = { - // TODO: reuse buffer? - buffer = new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE) + buffer.clear() } override def start(): Unit = { @@ -384,10 +374,8 @@ private[parquet] class CatalystArrayConverter( } } - // TODO: think about reusing the buffer override def end(): Unit = { assert(parent != null) - // TODO: use iterators if possible, avoid Row wrapping parent.updateField(index, new GenericRow(buffer.toArray)) clearBuffer() } @@ -396,20 +384,27 @@ private[parquet] class CatalystArrayConverter( override def getCurrentRecord: Row = throw new UnsupportedOperationException } -private[parquet] class CatalystNativeArrayConverter[T <: NativeType]( +/** + * 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.catalyst.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: NativeType, val index: Int, protected[parquet] val parent: CatalystConverter, protected[parquet] var capacity: Int = CatalystArrayConverter.INITIAL_ARRAY_SIZE) extends GroupConverter with CatalystConverter { - // similar comment as in [[Decoder]]: this should probably be in NativeType - private val classTag = { - val mirror = runtimeMirror(Utils.getSparkClassLoader) - ClassTag[T#JvmType](mirror.runtimeClass(elementType.tag.tpe)) - } + type nativeType = elementType.JvmType - private var buffer: Array[T#JvmType] = classTag.newArray(capacity) + private var buffer: Array[nativeType] = elementType.classTag.newArray(capacity) private var elements: Int = 0 @@ -432,43 +427,43 @@ private[parquet] class CatalystNativeArrayConverter[T <: NativeType]( // Overriden here to avoid auto-boxing for primitive types override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[T#JvmType] + buffer(elements) = value.asInstanceOf[nativeType] elements += 1 } override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[T#JvmType] + buffer(elements) = value.asInstanceOf[nativeType] elements += 1 } override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[T#JvmType] + buffer(elements) = value.asInstanceOf[nativeType] elements += 1 } override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[T#JvmType] + buffer(elements) = value.asInstanceOf[nativeType] elements += 1 } override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[T#JvmType] + buffer(elements) = value.asInstanceOf[nativeType] elements += 1 } override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { checkGrowBuffer() - buffer(elements) = value.getBytes.asInstanceOf[T#JvmType] + buffer(elements) = value.getBytes.asInstanceOf[nativeType] elements += 1 } override protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit = { checkGrowBuffer() - buffer(elements) = value.toStringUsingUTF8.asInstanceOf[T#JvmType] + buffer(elements) = value.toStringUsingUTF8.asInstanceOf[nativeType] elements += 1 } @@ -482,12 +477,7 @@ private[parquet] class CatalystNativeArrayConverter[T <: NativeType]( assert(parent != null) parent.updateField( index, - new GenericRow { - // TODO: it would be much nicer to use a view here but GenericRow requires an Array - // TODO: we should avoid using GenericRow as a wrapper but [[GetField]] current - // requires that - override val values = buffer.slice(0, elements).map(_.asInstanceOf[Any]) - }) + new NativeRow[nativeType](buffer.slice(0, elements))) clearBuffer() } @@ -497,7 +487,7 @@ private[parquet] class CatalystNativeArrayConverter[T <: NativeType]( private def checkGrowBuffer(): Unit = { if (elements >= capacity) { val newCapacity = 2 * capacity - val tmp: Array[T#JvmType] = classTag.newArray(newCapacity) + val tmp: Array[nativeType] = elementType.classTag.newArray(newCapacity) Array.copy(buffer, 0, tmp, 0, capacity) buffer = tmp capacity = newCapacity From 0ae9376039c718c89d2957e34e63fc8741385610 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 10 May 2014 21:23:30 +0300 Subject: [PATCH 28/48] Doc strings and simplifying ParquetConverter.scala --- .../spark/sql/parquet/ParquetConverter.scala | 101 +++++++++++++----- 1 file changed, 74 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 0463433a66ecd..66cd8a7f1d8b7 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 @@ -26,6 +26,31 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions.{NativeRow, GenericRow, Row, Attribute} import org.apache.spark.sql.parquet.CatalystConverter.FieldType +/** + * Collection of converters of Parquet types (Group and primitive types) that + * model arrays and maps. The convertions 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[parquet] object CatalystConverter { // The type internally used for fields type FieldType = StructField @@ -80,7 +105,7 @@ private[parquet] object CatalystConverter { val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) // For non-nested types we use the optimized Row converter if (attributes.forall(a => ParquetTypesConverter.isPrimitiveType(a.dataType))) { - new PrimitiveRowGroupConverter(attributes) + new CatalystPrimitiveRowConverter(attributes) } else { new CatalystGroupConverter(attributes) } @@ -88,16 +113,29 @@ private[parquet] object CatalystConverter { } private[parquet] trait CatalystConverter { - // the number of fields this group has + /** + * The number of fields this group has + */ protected[parquet] val size: Int - // the index of this converter in the parent + /** + * The index of this converter in the parent + */ protected[parquet] val index: Int - // the parent converter + /** + * The parent converter + */ protected[parquet] val parent: CatalystConverter - // for child converters to update upstream values + /** + * 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 = @@ -125,8 +163,12 @@ private[parquet] trait CatalystConverter { protected[parquet] def clearBuffer(): Unit - // Should be only called in root group converter! - def getCurrentRecord: Row + /** + * Should only be called in the root (group) converter! + * + * @return + */ + def getCurrentRecord: Row = throw new UnsupportedOperationException } /** @@ -152,7 +194,9 @@ private[parquet] class CatalystGroupConverter( buffer=new ArrayBuffer[Row]( CatalystArrayConverter.INITIAL_ARRAY_SIZE)) - // This constructor is used for the root converter only + /** + * This constructor is used for the root converter only! + */ def this(attributes: Seq[Attribute]) = this(attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), 0, null) @@ -163,8 +207,7 @@ private[parquet] class CatalystGroupConverter( override val size = schema.size - // Should be only called in root group converter! - def getCurrentRecord: Row = { + override def getCurrentRecord: Row = { 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 @@ -205,7 +248,7 @@ private[parquet] class CatalystGroupConverter( * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his * converter is optimized for rows of primitive types (non-nested records). */ -private[parquet] class PrimitiveRowGroupConverter( +private[parquet] class CatalystPrimitiveRowConverter( protected[parquet] val schema: Seq[FieldType], protected[parquet] var current: ParquetRelation.RowType) extends GroupConverter with CatalystConverter { @@ -228,7 +271,7 @@ private[parquet] class PrimitiveRowGroupConverter( override val parent = null // Should be only called in root group converter! - def getCurrentRecord: ParquetRelation.RowType = current + override def getCurrentRecord: ParquetRelation.RowType = current override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) @@ -379,9 +422,6 @@ private[parquet] class CatalystArrayConverter( parent.updateField(index, new GenericRow(buffer.toArray)) clearBuffer() } - - // Should be only called in root group converter! - override def getCurrentRecord: Row = throw new UnsupportedOperationException } /** @@ -481,9 +521,6 @@ private[parquet] class CatalystNativeArrayConverter( clearBuffer() } - // Should be only called in root group converter! - override def getCurrentRecord: Row = throw new UnsupportedOperationException - private def checkGrowBuffer(): Unit = { if (elements >= capacity) { val newCapacity = 2 * capacity @@ -495,8 +532,15 @@ private[parquet] class CatalystNativeArrayConverter( } } -// this is for multi-element groups of primitive or complex types -// that have repetition level optional or required (so struct fields) +/** + * 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: Seq[FieldType], override protected[parquet] val index: Int, @@ -511,11 +555,18 @@ private[parquet] class CatalystStructConverter( // TODO: use iterators if possible, avoid Row wrapping! parent.updateField(index, new GenericRow(current.toArray)) } - - // Should be only called in root group converter! - override def getCurrentRecord: Row = throw new UnsupportedOperationException } +/** + * 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.catalyst.types.MapType]]. + * + * @param schema + * @param index + * @param parent + */ private[parquet] class CatalystMapConverter( protected[parquet] val schema: Seq[FieldType], override protected[parquet] val index: Int, @@ -557,7 +608,6 @@ private[parquet] class CatalystMapConverter( } override protected[parquet] def clearBuffer(): Unit = {} - override def getCurrentRecord: Row = throw new UnsupportedOperationException } override protected[parquet] val size: Int = 1 @@ -568,15 +618,12 @@ private[parquet] class CatalystMapConverter( map.clear() } - // TODO: think about reusing the buffer override def end(): Unit = { parent.updateField(index, map.toMap) } override def getConverter(fieldIndex: Int): Converter = keyValueConverter - override def getCurrentRecord: Row = throw new UnsupportedOperationException - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = throw new UnsupportedOperationException } From 32229c755f1d04bfa2851aab94f7412dcee4d5d9 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 11 May 2014 21:28:05 +0300 Subject: [PATCH 29/48] Removing Row nested values and placing by generic types --- .../spark/sql/catalyst/expressions/Row.scala | 62 -------- .../catalyst/expressions/complexTypes.scala | 4 +- .../spark/sql/parquet/ParquetConverter.scala | 38 +++-- .../sql/parquet/ParquetTableSupport.scala | 29 +++- .../spark/sql/parquet/ParquetQuerySuite.scala | 146 +++++++++++------- 5 files changed, 136 insertions(+), 143 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index b587610749a15..71825eca73b80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -206,68 +206,6 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { override def copy() = new GenericRow(values.clone()) } -// TODO: this is an awful lot of code duplication. If values would be covariant we could reuse -// much of GenericRow -class NativeRow[T](protected[catalyst] val values: Array[T]) extends Row { - - /** No-arg constructor for serialization. */ - def this() = this(null) - - def this(elementType: NativeType, size: Int) = - this(elementType.classTag.newArray(size).asInstanceOf[Array[T]]) - - def iterator = values.iterator - - def length = values.length - - def apply(i: Int) = values(i) - - def isNullAt(i: Int) = values(i) == null - - def getInt(i: Int): Int = { - if (values(i) == null) sys.error("Failed to check null bit for primitive int value.") - values(i).asInstanceOf[Int] - } - - def getLong(i: Int): Long = { - if (values(i) == null) sys.error("Failed to check null bit for primitive long value.") - values(i).asInstanceOf[Long] - } - - def getDouble(i: Int): Double = { - if (values(i) == null) sys.error("Failed to check null bit for primitive double value.") - values(i).asInstanceOf[Double] - } - - def getFloat(i: Int): Float = { - if (values(i) == null) sys.error("Failed to check null bit for primitive float value.") - values(i).asInstanceOf[Float] - } - - def getBoolean(i: Int): Boolean = { - if (values(i) == null) sys.error("Failed to check null bit for primitive boolean value.") - values(i).asInstanceOf[Boolean] - } - - def getShort(i: Int): Short = { - if (values(i) == null) sys.error("Failed to check null bit for primitive short value.") - values(i).asInstanceOf[Short] - } - - def getByte(i: Int): Byte = { - if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.") - values(i).asInstanceOf[Byte] - } - - def getString(i: Int): String = { - if (values(i) == null) sys.error("Failed to check null bit for primitive String value.") - values(i).asInstanceOf[String] - } - - def copy() = this -} - - class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = this(ordering.map(BindReferences.bindReference(_, inputSchema))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index b6aeae92f8bec..285118fc81ded 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -50,7 +50,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { null } else { if (child.dataType.isInstanceOf[ArrayType]) { - val baseValue = value.asInstanceOf[Seq[_]] + val baseValue = value.asInstanceOf[Array[_]] val o = key.asInstanceOf[Int] if (o >= baseValue.size || o < 0) { null @@ -92,7 +92,7 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] override def eval(input: Row): Any = { - val baseValue = child.eval(input).asInstanceOf[Row] + val baseValue = child.eval(input).asInstanceOf[Seq[_]] if (baseValue == null) null else baseValue(ordinal) } 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 66cd8a7f1d8b7..8e13b5b72509f 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 @@ -23,12 +23,12 @@ import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.expressions.{NativeRow, GenericRow, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} import org.apache.spark.sql.parquet.CatalystConverter.FieldType /** - * Collection of converters of Parquet types (Group and primitive types) that - * model arrays and maps. The convertions are partly based on the AvroParquet + * 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. * @@ -51,7 +51,7 @@ import org.apache.spark.sql.parquet.CatalystConverter.FieldType *
    */ -private[parquet] object CatalystConverter { +private[sql] object CatalystConverter { // The type internally used for fields type FieldType = StructField @@ -63,6 +63,10 @@ private[parquet] object CatalystConverter { val MAP_VALUE_SCHEMA_NAME = "value" val MAP_SCHEMA_NAME = "map" + type ArrayScalaType[T] = Array[T] + type StructScalaType[T] = Seq[T] + type MapScalaType[K, V] = Map[K, V] + protected[parquet] def createConverter( field: FieldType, fieldIndex: Int, @@ -325,7 +329,6 @@ private[parquet] class CatalystPrimitiveRowConverter( private[parquet] class CatalystPrimitiveConverter( parent: CatalystConverter, fieldIndex: Int) extends PrimitiveConverter { - // TODO: consider refactoring these together with ParquetTypesConverter override def addBinary(value: Binary): Unit = parent.updateBinary(fieldIndex, value) @@ -404,6 +407,9 @@ private[parquet] class CatalystArrayConverter( 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 } @@ -419,7 +425,8 @@ private[parquet] class CatalystArrayConverter( override def end(): Unit = { assert(parent != null) - parent.updateField(index, new GenericRow(buffer.toArray)) + // here we need to make sure to use ArrayScalaType + parent.updateField(index, buffer.toArray) clearBuffer() } } @@ -444,7 +451,8 @@ private[parquet] class CatalystNativeArrayConverter( type nativeType = elementType.JvmType - private var buffer: Array[nativeType] = elementType.classTag.newArray(capacity) + private var buffer: CatalystConverter.ArrayScalaType[nativeType] = + elementType.classTag.newArray(capacity) private var elements: Int = 0 @@ -515,16 +523,18 @@ private[parquet] class CatalystNativeArrayConverter( override def end(): Unit = { assert(parent != null) + // here we need to make sure to use ArrayScalaType parent.updateField( index, - new NativeRow[nativeType](buffer.slice(0, elements))) + buffer.slice(0, elements)) clearBuffer() } private def checkGrowBuffer(): Unit = { if (elements >= capacity) { val newCapacity = 2 * capacity - val tmp: Array[nativeType] = elementType.classTag.newArray(newCapacity) + val tmp: CatalystConverter.ArrayScalaType[nativeType] = + elementType.classTag.newArray(newCapacity) Array.copy(buffer, 0, tmp, 0, capacity) buffer = tmp capacity = newCapacity @@ -552,8 +562,10 @@ private[parquet] class CatalystStructConverter( // TODO: think about reusing the buffer override def end(): Unit = { assert(!isRootConverter) - // TODO: use iterators if possible, avoid Row wrapping! - parent.updateField(index, new GenericRow(current.toArray)) + // 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, current.toArray.toSeq) } } @@ -619,6 +631,7 @@ private[parquet] class CatalystMapConverter( } override def end(): Unit = { + // here we need to make sure to use MapScalaType parent.updateField(index, map.toMap) } @@ -627,6 +640,3 @@ private[parquet] class CatalystMapConverter( 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 b50c51c0a8b2f..f839fdc961df4 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 @@ -140,9 +140,15 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] def writeValue(schema: DataType, value: Any): Unit = { if (value != null && value != Nil) { schema match { - case t @ ArrayType(_) => writeArray(t, value.asInstanceOf[Row]) - case t @ MapType(_, _) => writeMap(t, value.asInstanceOf[Map[Any, Any]]) - case t @ StructType(_) => writeStruct(t, value.asInstanceOf[Row]) + case t @ ArrayType(_) => writeArray( + t, + value.asInstanceOf[CatalystConverter.ArrayScalaType[_]]) + case t @ MapType(_, _) => writeMap( + t, + value.asInstanceOf[CatalystConverter.MapScalaType[_, _]]) + case t @ StructType(_) => writeStruct( + t, + value.asInstanceOf[CatalystConverter.StructScalaType[_]]) case _ => writePrimitive(schema.asInstanceOf[PrimitiveType], value) } } @@ -166,7 +172,9 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } } - private[parquet] def writeStruct(schema: StructType, struct: Row): Unit = { + private[parquet] def writeStruct( + schema: StructType, + struct: CatalystConverter.StructScalaType[_]): Unit = { if (struct != null && struct != Nil) { val fields = schema.fields.toArray writer.startGroup() @@ -183,7 +191,11 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } } - private[parquet] def writeArray(schema: ArrayType, array: Row): Unit = { + // TODO: support null values, see + // https://issues.apache.org/jira/browse/SPARK-1649 + private[parquet] def writeArray( + schema: ArrayType, + array: CatalystConverter.ArrayScalaType[_]): Unit = { val elementType = schema.elementType writer.startGroup() if (array.size > 0) { @@ -198,8 +210,11 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { writer.endGroup() } - // TODO: this does not allow null values! Should these be supported? - private[parquet] def writeMap(schema: MapType, map: Map[_, _]): Unit = { + // TODO: support null values, see + // https://issues.apache.org/jira/browse/SPARK-1649 + private[parquet] def writeMap( + schema: MapType, + map: CatalystConverter.MapScalaType[_, _]): Unit = { writer.startGroup() if (map.size > 0) { writer.startField(CatalystConverter.MAP_SCHEMA_NAME, 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index a42d2cbb1f855..39a4e25ae7ae3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -399,7 +399,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Importing nested Parquet file (Addressbook)") { - implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val result = TestSQLContext .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD @@ -408,23 +407,32 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result.size === 2) val first_record = result(0) val second_record = result(1) - val first_owner_numbers = result(0)(1) - val first_contacts = result(0)(2) + assert(first_record != null) + assert(second_record != null) assert(first_record.size === 3) assert(second_record(1) === null) assert(second_record(2) === null) assert(second_record(0) === "A. Nonymous") assert(first_record(0) === "Julien Le Dem") + val first_owner_numbers = first_record(1) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]] + val first_contacts = first_record(2) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]] + assert(first_owner_numbers != null) assert(first_owner_numbers(0) === "555 123 4567") assert(first_owner_numbers(2) === "XXX XXX XXXX") - assert(first_contacts(0).size === 2) - assert(first_contacts(0)(0) === "Dmitriy Ryaboy") - assert(first_contacts(0)(1) === "555 987 6543") - assert(first_contacts(1)(0) === "Chris Aniszczyk") + assert(first_contacts(0) + .asInstanceOf[CatalystConverter.StructScalaType[_]].size === 2) + val first_contacts_entry_one = first_contacts(0) + .asInstanceOf[CatalystConverter.StructScalaType[_]] + assert(first_contacts_entry_one(0) === "Dmitriy Ryaboy") + assert(first_contacts_entry_one(1) === "555 987 6543") + val first_contacts_entry_two = first_contacts(1) + .asInstanceOf[CatalystConverter.StructScalaType[_]] + assert(first_contacts_entry_two(0) === "Chris Aniszczyk") } test("Importing nested Parquet file (nested numbers)") { - implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val result = TestSQLContext .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD @@ -433,19 +441,27 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(0).size === 5, "number of fields in row incorrect") assert(result(0)(0) === 1) assert(result(0)(1) === 7) - assert(result(0)(2).size === 3) - assert(result(0)(2)(0) === (1.toLong << 32)) - assert(result(0)(2)(1) === (1.toLong << 33)) - assert(result(0)(2)(2) === (1.toLong << 34)) - assert(result(0)(3)(0).size === 2) - assert(result(0)(3)(0)(0) === 2.5) - assert(result(0)(3)(0)(1) === false) - assert(result(0)(4).size === 2) - assert(result(0)(4)(0).size === 2) - assert(result(0)(4)(1).size === 1) - assert(result(0)(4)(0)(0)(0) === 7) - assert(result(0)(4)(0)(1)(0) === 8) - assert(result(0)(4)(1)(0)(0) === 9) + val subresult1 = result(0)(2).asInstanceOf[CatalystConverter.ArrayScalaType[_]] + assert(subresult1.size === 3) + assert(subresult1(0) === (1.toLong << 32)) + assert(subresult1(1) === (1.toLong << 33)) + assert(subresult1(2) === (1.toLong << 34)) + val subresult2 = result(0)(3) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) + .asInstanceOf[CatalystConverter.StructScalaType[_]] + assert(subresult2.size === 2) + assert(subresult2(0) === 2.5) + assert(subresult2(1) === false) + val subresult3 = result(0)(4) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]] + assert(subresult3.size === 2) + assert(subresult3(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size === 2) + val subresult4 = subresult3(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]] + assert(subresult4(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 7) + assert(subresult4(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 8) + assert(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size === 1) + assert(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 9) } test("Simple query on addressbook") { @@ -458,7 +474,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Projection in addressbook") { - implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD @@ -473,7 +488,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Simple query on nested int data") { - implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD @@ -484,17 +498,23 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result1(0)(0) === 2.5) val result2 = sql("SELECT entries[0] FROM data").collect() assert(result2.size === 1) - assert(result2(0)(0).size === 2) - assert(result2(0)(0)(0) === 2.5) - assert(result2(0)(0)(1) === false) + val subresult1 = result2(0)(0).asInstanceOf[CatalystConverter.StructScalaType[_]] + assert(subresult1.size === 2) + assert(subresult1(0) === 2.5) + assert(subresult1(1) === false) val result3 = sql("SELECT outerouter FROM data").collect() - assert(result3(0)(0)(0)(0)(0) === 7) - assert(result3(0)(0)(0)(1)(0) === 8) - assert(result3(0)(0)(1)(0)(0) === 9) + val subresult2 = result3(0)(0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]] + assert(subresult2(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 7) + assert(subresult2(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 8) + assert(result3(0)(0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](1) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 9) } test("nested structs") { - implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir3.toString) .toSchemaRDD @@ -514,32 +534,38 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("simple map") { - implicit def anyToMap(value: Any) = value.asInstanceOf[Map[String, Int]] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD data.registerAsTable("mapTable") val result1 = sql("SELECT data1 FROM mapTable").collect() assert(result1.size === 1) - assert(result1(0)(0).toMap.getOrElse("key1", 0) === 1) - assert(result1(0)(0).toMap.getOrElse("key2", 0) === 2) + assert(result1(0)(0) + .asInstanceOf[CatalystConverter.MapScalaType[String, _]] + .getOrElse("key1", 0) === 1) + assert(result1(0)(0) + .asInstanceOf[CatalystConverter.MapScalaType[String, _]] + .getOrElse("key2", 0) === 2) val result2 = sql("SELECT data1[key1] FROM mapTable").collect() assert(result2(0)(0) === 1) } test("map with struct values") { - implicit def anyToMap(value: Any) = value.asInstanceOf[Map[String, Row]] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD data.registerAsTable("mapTable") val result1 = sql("SELECT data2 FROM mapTable").collect() assert(result1.size === 1) - val entry1 = result1(0)(0).getOrElse("7", null) + val entry1 = result1(0)(0) + .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] + .getOrElse("7", null) assert(entry1 != null) assert(entry1(0) === 42) assert(entry1(1) === "the answer") - val entry2 = result1(0)(0).getOrElse("8", null) + val entry2 = result1(0)(0) + .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] + .getOrElse("8", null) assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) @@ -552,7 +578,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Writing out Addressbook and reading it back in") { // TODO: find out why CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME // has no effect in this test case - implicit def anyToRow(value: Any): Row = value.asInstanceOf[Row] val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) val result = TestSQLContext @@ -574,7 +599,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Writing out Map and reading it back in") { - implicit def anyToMap(value: Any) = value.asInstanceOf[Map[String, Row]] val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD @@ -590,11 +614,15 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result1(0)(0) === 2) val result2 = sql("SELECT data2 FROM tmpmapcopy").collect() assert(result2.size === 1) - val entry1 = result2(0)(0).getOrElse("7", null) + val entry1 = result2(0)(0) + .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] + .getOrElse("7", null) assert(entry1 != null) assert(entry1(0) === 42) assert(entry1(1) === "the answer") - val entry2 = result2(0)(0).getOrElse("8", null) + val entry2 = result2(0)(0) + .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] + .getOrElse("8", null) assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) @@ -705,39 +733,41 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(resultPrimitives(0)(1) === 0.5) assert(resultPrimitives(0)(2) === "foo") val resultPrimitiveArray = sql("SELECT testPrimitiveArray FROM avroTable").collect() - assert(resultPrimitiveArray(0)(0).asInstanceOf[Row](0) === 1) - assert(resultPrimitiveArray(0)(0).asInstanceOf[Row](1) === 2) - assert(resultPrimitiveArray(0)(0).asInstanceOf[Row](2) === 3) + assert(resultPrimitiveArray(0)(0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 1) + assert(resultPrimitiveArray(0)(0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](1) === 2) + assert(resultPrimitiveArray(0)(0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](2) === 3) val resultComplexArray = sql("SELECT testComplexArray FROM avroTable").collect() - assert(resultComplexArray(0)(0).asInstanceOf[Row].size === 2) + assert(resultComplexArray(0)(0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]].size === 2) assert( resultComplexArray(0)(0) - .asInstanceOf[Row] - .apply(0) - .asInstanceOf[Map[String, String]].get("key11").get.equals("data11")) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) + .asInstanceOf[CatalystConverter.MapScalaType[String, String]] + .get("key11").get.equals("data11")) assert( resultComplexArray(0)(0) - .asInstanceOf[Row] - .apply(1) - .asInstanceOf[Map[String, String]].get("key22").get.equals("data22")) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](1) + .asInstanceOf[CatalystConverter.MapScalaType[String, String]] + .get("key22").get.equals("data22")) val resultPrimitiveMap = sql("SELECT testPrimitiveMap FROM avroTable").collect() assert( resultPrimitiveMap(0)(0) - .asInstanceOf[Map[String, Boolean]].get("key1").get === true) + .asInstanceOf[CatalystConverter.MapScalaType[String, Boolean]].get("key1").get === true) assert( resultPrimitiveMap(0)(0) - .asInstanceOf[Map[String, Boolean]].get("key2").get === false) + .asInstanceOf[CatalystConverter.MapScalaType[String, Boolean]].get("key2").get === false) val resultComplexMap = sql("SELECT testComplexMap FROM avroTable").collect() val mapResult1 = resultComplexMap(0)(0) - .asInstanceOf[Map[String, Row]] - .get("compKey1") - .get + .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.ArrayScalaType[_]]] + .get("compKey1").get val mapResult2 = resultComplexMap(0)(0) - .asInstanceOf[Map[String, Row]] - .get("compKey2") - .get + .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.ArrayScalaType[_]]] + .get("compKey2").get assert(mapResult1(0) === 0.1f) assert(mapResult1(2) === 0.3f) assert(mapResult2(0) === 1.1f) From cbb579346a7b39d1c3797acda5d861577abd7703 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 11 May 2014 21:40:04 +0300 Subject: [PATCH 30/48] Code review feedback --- .../spark/sql/parquet/ParquetConverter.scala | 20 +++++++++---------- .../spark/sql/parquet/ParquetRelation.scala | 2 +- .../sql/parquet/ParquetTableOperations.scala | 2 +- .../spark/sql/parquet/ParquetTypes.scala | 2 +- 4 files changed, 13 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 8e13b5b72509f..f694e59252fe3 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 @@ -449,9 +449,9 @@ private[parquet] class CatalystNativeArrayConverter( protected[parquet] var capacity: Int = CatalystArrayConverter.INITIAL_ARRAY_SIZE) extends GroupConverter with CatalystConverter { - type nativeType = elementType.JvmType + type NativeType = elementType.JvmType - private var buffer: CatalystConverter.ArrayScalaType[nativeType] = + private var buffer: CatalystConverter.ArrayScalaType[NativeType] = elementType.classTag.newArray(capacity) private var elements: Int = 0 @@ -475,43 +475,43 @@ private[parquet] class CatalystNativeArrayConverter( // Overriden here to avoid auto-boxing for primitive types override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[nativeType] + buffer(elements) = value.asInstanceOf[NativeType] elements += 1 } override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[nativeType] + buffer(elements) = value.asInstanceOf[NativeType] elements += 1 } override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[nativeType] + buffer(elements) = value.asInstanceOf[NativeType] elements += 1 } override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[nativeType] + buffer(elements) = value.asInstanceOf[NativeType] elements += 1 } override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[nativeType] + buffer(elements) = value.asInstanceOf[NativeType] elements += 1 } override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { checkGrowBuffer() - buffer(elements) = value.getBytes.asInstanceOf[nativeType] + buffer(elements) = value.getBytes.asInstanceOf[NativeType] elements += 1 } override protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit = { checkGrowBuffer() - buffer(elements) = value.toStringUsingUTF8.asInstanceOf[nativeType] + buffer(elements) = value.toStringUsingUTF8.asInstanceOf[NativeType] elements += 1 } @@ -533,7 +533,7 @@ private[parquet] class CatalystNativeArrayConverter( private def checkGrowBuffer(): Unit = { if (elements >= capacity) { val newCapacity = 2 * capacity - val tmp: CatalystConverter.ArrayScalaType[nativeType] = + val tmp: CatalystConverter.ArrayScalaType[NativeType] = elementType.classTag.newArray(newCapacity) Array.copy(buffer, 0, tmp, 0, capacity) buffer = tmp diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 84ccd8ccc9661..e581fb0ae71c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -151,7 +151,7 @@ private[sql] object ParquetRelation { } if (fs.exists(path) && - !fs.getFileStatus(path) + !fs.getFileStatus(path) .getPermission .getUserAction .implies(FsAction.READ_WRITE)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 3e14ec31fd5f7..b11888af45853 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -168,7 +168,7 @@ case class InsertIntoParquetTable( val writeSupport = if (child.output.map(_.dataType).forall(_.isPrimitive())) { - logger.info("Initializing MutableRowWriteSupport") + logger.debug("Initializing MutableRowWriteSupport") classOf[org.apache.spark.sql.parquet.MutableRowWriteSupport] } else { classOf[org.apache.spark.sql.parquet.RowWriteSupport] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index c2e1d918f96e3..5843f68ed41d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -176,7 +176,7 @@ private[parquet] object ParquetTypesConverter { /** * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into - * the corrponsing Parquet `Type`. + * the corresponding Parquet `Type`. * * The conversion follows the rules below: *
      From 191bc0d18e721251b1a79390533199794a56ea21 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 24 May 2014 11:49:09 +0300 Subject: [PATCH 31/48] Changing to Seq for ArrayType, refactoring SQLParser for nested field extension --- .../apache/spark/sql/catalyst/SqlParser.scala | 112 +++++++++--------- .../catalyst/expressions/complexTypes.scala | 4 +- .../catalyst/plans/logical/LogicalPlan.scala | 76 ++---------- .../spark/sql/parquet/ParquetConverter.scala | 13 +- .../spark/sql/parquet/ParquetTestData.scala | 4 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 97 ++++++++++----- 6 files changed, 144 insertions(+), 162 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index d2baf09074799..2ad2d04af5704 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -66,43 +66,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected case class Keyword(str: String) protected implicit def asParser(k: Keyword): Parser[String] = - allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) - - protected class SqlLexical extends StdLexical { - case class FloatLit(chars: String) extends Token { - override def toString = chars - } - override lazy val token: Parser[Token] = ( - identChar ~ rep( identChar | digit ) ^^ - { case first ~ rest => processIdent(first :: rest mkString "") } - | rep1(digit) ~ opt('.' ~> rep(digit)) ^^ { - case i ~ None => NumericLit(i mkString "") - case i ~ Some(d) => FloatLit(i.mkString("") + "." + d.mkString("")) - } - | '\'' ~ rep( chrExcept('\'', '\n', EofCh) ) ~ '\'' ^^ - { case '\'' ~ chars ~ '\'' => StringLit(chars mkString "") } - | '\"' ~ rep( chrExcept('\"', '\n', EofCh) ) ~ '\"' ^^ - { case '\"' ~ chars ~ '\"' => StringLit(chars mkString "") } - | EofCh ^^^ EOF - | '\'' ~> failure("unclosed string literal") - | '\"' ~> failure("unclosed string literal") - | delim - | failure("illegal character") - ) - - override def identChar = letter | elem('.') | elem('_') | elem('[') | elem(']') - - override def whitespace: Parser[Any] = rep( - whitespaceChar - | '/' ~ '*' ~ comment - | '/' ~ '/' ~ rep( chrExcept(EofCh, '\n') ) - | '#' ~ rep( chrExcept(EofCh, '\n') ) - | '-' ~ '-' ~ rep( chrExcept(EofCh, '\n') ) - | '/' ~ '*' ~ failure("unclosed comment") - ) - } - - override val lexical = new SqlLexical + lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) protected val ALL = Keyword("ALL") protected val AND = Keyword("AND") @@ -161,24 +125,9 @@ class SqlParser extends StandardTokenParsers with PackratParsers { this.getClass .getMethods .filter(_.getReturnType == classOf[Keyword]) - .map(_.invoke(this).asInstanceOf[Keyword]) - - /** Generate all variations of upper and lower case of a given string */ - private def allCaseVersions(s: String, prefix: String = ""): Stream[String] = { - if (s == "") { - Stream(prefix) - } else { - allCaseVersions(s.tail, prefix + s.head.toLower) ++ - allCaseVersions(s.tail, prefix + s.head.toUpper) - } - } + .map(_.invoke(this).asInstanceOf[Keyword].str) - lexical.reserved ++= reservedWords.flatMap(w => allCaseVersions(w.str)) - - lexical.delimiters += ( - "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":", "[", "]" - ) + override val lexical = new SqlLexical(reservedWords) protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = { exprs.zipWithIndex.map { @@ -383,14 +332,13 @@ class SqlParser extends StandardTokenParsers with PackratParsers { elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) protected lazy val baseExpression: PackratParser[Expression] = - expression ~ "[" ~ expression <~ "]" ^^ { + expression ~ "[" ~ expression <~ "]" ^^ { case base ~ _ ~ ordinal => GetItem(base, ordinal) } | TRUE ^^^ Literal(true, BooleanType) | FALSE ^^^ Literal(false, BooleanType) | cast | "(" ~> expression <~ ")" | - "[" ~> literal <~ "]" | function | "-" ~> literal ^^ UnaryMinus | ident ^^ UnresolvedAttribute | @@ -400,3 +348,55 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected lazy val dataType: Parser[DataType] = STRING ^^^ StringType } + +class SqlLexical(val keywords: Seq[String]) extends StdLexical { + case class FloatLit(chars: String) extends Token { + override def toString = chars + } + + reserved ++= keywords.flatMap(w => allCaseVersions(w)) + + delimiters += ( + "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", + ",", ";", "%", "{", "}", ":", "[", "]" + ) + + override lazy val token: Parser[Token] = ( + identChar ~ rep( identChar | digit ) ^^ + { case first ~ rest => processIdent(first :: rest mkString "") } + | rep1(digit) ~ opt('.' ~> rep(digit)) ^^ { + case i ~ None => NumericLit(i mkString "") + case i ~ Some(d) => FloatLit(i.mkString("") + "." + d.mkString("")) + } + | '\'' ~ rep( chrExcept('\'', '\n', EofCh) ) ~ '\'' ^^ + { case '\'' ~ chars ~ '\'' => StringLit(chars mkString "") } + | '\"' ~ rep( chrExcept('\"', '\n', EofCh) ) ~ '\"' ^^ + { case '\"' ~ chars ~ '\"' => StringLit(chars mkString "") } + | EofCh ^^^ EOF + | '\'' ~> failure("unclosed string literal") + | '\"' ~> failure("unclosed string literal") + | delim + | failure("illegal character") + ) + + override def identChar = letter | elem('_') | elem('.') + + override def whitespace: Parser[Any] = rep( + whitespaceChar + | '/' ~ '*' ~ comment + | '/' ~ '/' ~ rep( chrExcept(EofCh, '\n') ) + | '#' ~ rep( chrExcept(EofCh, '\n') ) + | '-' ~ '-' ~ rep( chrExcept(EofCh, '\n') ) + | '/' ~ '*' ~ failure("unclosed comment") + ) + + /** Generate all variations of upper and lower case of a given string */ + def allCaseVersions(s: String, prefix: String = ""): Stream[String] = { + if (s == "") { + Stream(prefix) + } else { + allCaseVersions(s.tail, prefix + s.head.toLower) ++ + allCaseVersions(s.tail, prefix + s.head.toUpper) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 285118fc81ded..37ccb965feb87 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -50,7 +50,9 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { null } else { if (child.dataType.isInstanceOf[ArrayType]) { - val baseValue = value.asInstanceOf[Array[_]] + // TODO: consider using Array[_] for ArrayType child to avoid + // boxing of primitives + val baseValue = value.asInstanceOf[Seq[_]] val o = key.asInstanceOf[Int] if (o >= baseValue.size || o < 0) { null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 7ff2287fbee03..76459f49cae02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -58,53 +58,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { * can contain ordinal expressions, such as `field[i][j][k]...`. */ def resolve(name: String): Option[NamedExpression] = { - def expandFunc(expType: (Expression, DataType), field: String): (Expression, DataType) = { - val (exp, t) = expType - val ordinalRegExp = """(\[(\d+|\w+)\])""".r - val fieldName = if (ordinalRegExp.findFirstIn(field).isDefined) { - field.substring(0, field.indexOf("[")) - } else { - field - } - t match { - case ArrayType(elementType) => - val ordinals = ordinalRegExp - .findAllIn(field) - .matchData - .map(_.group(2)) - (ordinals.foldLeft(exp)((v1: Expression, v2: String) => - GetItem(v1, Literal(v2.toInt))), elementType) - case MapType(keyType, valueType) => - val ordinals = ordinalRegExp - .findAllIn(field) - .matchData - .map(_.group(2)) - // TODO: we should recover the JVM type of keyType to match the - // actual type of the key?! should we restrict ourselves to NativeType? - (ordinals.foldLeft(exp)((v1: Expression, v2: String) => - GetItem(v1, Literal(v2, keyType))), valueType) - case StructType(fields) => - val structField = fields - .find(_.name == fieldName) - if (!structField.isDefined) { - throw new TreeNodeException( - this, s"Trying to resolve Attribute but field ${fieldName} is not defined") - } - structField.get.dataType match { - case ArrayType(elementType) => - val ordinals = ordinalRegExp.findAllIn(field).matchData.map(_.group(2)) - (ordinals.foldLeft( - GetField(exp, fieldName).asInstanceOf[Expression])((v1: Expression, v2: String) => - GetItem(v1, Literal(v2.toInt))), - elementType) - case _ => - (GetField(exp, fieldName), structField.get.dataType) - } - case _ => - expType - } - } - + // TODO: extend SqlParser to handle field expressions val parts = name.split("\\.") // Collect all attributes that are output by this nodes children where either the first part // matches the name or where the first part matches the scope and the second part matches the @@ -124,33 +78,21 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { remainingParts.head } if (option.name == relevantRemaining) (option, remainingParts.tail.toList) :: Nil else Nil*/ + // If the first part of the desired name matches a qualifier for this possible match, drop it. + /* TODO: from rebase! + val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts + if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil + */ } options.distinct match { - case (a, Nil) :: Nil => { - a.dataType match { - case ArrayType(_) | MapType(_, _) => - val expression = expandFunc((a: Expression, a.dataType), name)._1 - Some(Alias(expression, name)()) - case _ => Some(a) - } - } // One match, no nested fields, use it. + case (a, Nil) :: Nil => Some(a) // One match, no nested fields, use it. // One match, but we also need to extract the requested nested field. case (a, nestedFields) :: Nil => a.dataType match { case StructType(fields) => - // this is compatibility reasons with earlier code! - // TODO: why only nestedFields and not parts? - // check for absence of nested arrays so there are only fields - if ((parts(0) :: nestedFields).forall(!_.matches("\\w*\\[(\\d+|\\w+)\\]+"))) { - Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) - } else { - val expression = parts.foldLeft((a: Expression, a.dataType))(expandFunc)._1 - Some(Alias(expression, nestedFields.last)()) - } - case _ => - val expression = parts.foldLeft((a: Expression, a.dataType))(expandFunc)._1 - Some(Alias(expression, nestedFields.last)()) + Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) + case _ => None // Don't know how to resolve these field references } case Nil => None // No matches. case ambiguousReferences => 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 f694e59252fe3..27c4c2ac76487 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 @@ -63,7 +63,8 @@ private[sql] object CatalystConverter { val MAP_VALUE_SCHEMA_NAME = "value" val MAP_SCHEMA_NAME = "map" - type ArrayScalaType[T] = Array[T] + // TODO: consider using Array[T] for arrays to avoid boxing of primitive types + type ArrayScalaType[T] = Seq[T] type StructScalaType[T] = Seq[T] type MapScalaType[K, V] = Map[K, V] @@ -426,7 +427,7 @@ private[parquet] class CatalystArrayConverter( override def end(): Unit = { assert(parent != null) // here we need to make sure to use ArrayScalaType - parent.updateField(index, buffer.toArray) + parent.updateField(index, buffer.toArray.toSeq) clearBuffer() } } @@ -451,8 +452,7 @@ private[parquet] class CatalystNativeArrayConverter( type NativeType = elementType.JvmType - private var buffer: CatalystConverter.ArrayScalaType[NativeType] = - elementType.classTag.newArray(capacity) + private var buffer: Array[NativeType] = elementType.classTag.newArray(capacity) private var elements: Int = 0 @@ -526,15 +526,14 @@ private[parquet] class CatalystNativeArrayConverter( // here we need to make sure to use ArrayScalaType parent.updateField( index, - buffer.slice(0, elements)) + buffer.slice(0, elements).toSeq) clearBuffer() } private def checkGrowBuffer(): Unit = { if (elements >= capacity) { val newCapacity = 2 * capacity - val tmp: CatalystConverter.ArrayScalaType[NativeType] = - elementType.classTag.newArray(newCapacity) + val tmp: Array[NativeType] = elementType.classTag.newArray(newCapacity) Array.copy(buffer, 0, tmp, 0, capacity) buffer = tmp capacity = newCapacity diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index d3449a6cbf77e..a11e19f3b6e63 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -377,13 +377,13 @@ private[sql] object ParquetTestData { val map2 = r1.addGroup(2) val keyValue3 = map2.addGroup(0) // TODO: currently only string key type supported - keyValue3.add(0, "7") + keyValue3.add(0, "seven") val valueGroup1 = keyValue3.addGroup(1) valueGroup1.add(0, 42.toLong) valueGroup1.add(1, "the answer") val keyValue4 = map2.addGroup(0) // TODO: currently only string key type supported - keyValue4.add(0, "8") + keyValue4.add(0, "eight") val valueGroup2 = keyValue4.addGroup(1) valueGroup2.add(0, 49.toLong) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 39a4e25ae7ae3..3cf7b0f10d09e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -32,14 +32,16 @@ import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil import parquet.schema.MessageTypeParser +import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.IntegerType +import org.apache.spark.sql.catalyst.types.{BooleanType, IntegerType} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.TestData import org.apache.spark.sql.SchemaRDD import org.apache.spark.sql.catalyst.util.getTempFilePath -import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.{SqlLexical, SqlParser} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, Star} import org.apache.spark.util.Utils // Implicits @@ -71,7 +73,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA var testRDD: SchemaRDD = null + // TODO: remove this once SqlParser can parse nested select statements + var nestedParserSqlContext: NestedParserSQLContext = null + override def beforeAll() { + nestedParserSqlContext = new NestedParserSQLContext(TestSQLContext.sparkContext) ParquetTestData.writeFile() ParquetTestData.writeFilterFile() ParquetTestData.writeNestedFile1() @@ -221,7 +227,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val source_rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) .map(i => TestRDDEntry(i, s"val_$i")) source_rdd.registerAsTable("source") - val dest_rdd = createParquetFile(dirname.toString, ("key", IntegerType), ("value", StringType)) + val dest_rdd = createParquetFile[TestRDDEntry](dirname.toString) dest_rdd.registerAsTable("dest") sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() val rdd_copy1 = sql("SELECT * FROM dest").collect() @@ -474,11 +480,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Projection in addressbook") { - val data = TestSQLContext + val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD data.registerAsTable("data") - val tmp = sql("SELECT owner, contacts[1].name FROM data").collect() + val query = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM data") + val tmp = query.collect() assert(tmp.size === 2) assert(tmp(0).size === 2) assert(tmp(0)(0) === "Julien Le Dem") @@ -488,21 +495,21 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Simple query on nested int data") { - val data = TestSQLContext + val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD data.registerAsTable("data") - val result1 = sql("SELECT entries[0].value FROM data").collect() + val result1 = nestedParserSqlContext.sql("SELECT entries[0].value FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) assert(result1(0)(0) === 2.5) - val result2 = sql("SELECT entries[0] FROM data").collect() + val result2 = nestedParserSqlContext.sql("SELECT entries[0] FROM data").collect() assert(result2.size === 1) val subresult1 = result2(0)(0).asInstanceOf[CatalystConverter.StructScalaType[_]] assert(subresult1.size === 2) assert(subresult1(0) === 2.5) assert(subresult1(1) === false) - val result3 = sql("SELECT outerouter FROM data").collect() + val result3 = nestedParserSqlContext.sql("SELECT outerouter FROM data").collect() val subresult2 = result3(0)(0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] @@ -515,19 +522,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("nested structs") { - val data = TestSQLContext + val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir3.toString) .toSchemaRDD data.registerAsTable("data") - val result1 = sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() + val result1 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) assert(result1(0)(0) === false) - val result2 = sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() + val result2 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() assert(result2.size === 1) assert(result2(0).size === 1) assert(result2(0)(0) === true) - val result3 = sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() + val result3 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() assert(result3.size === 1) assert(result3(0).size === 1) assert(result3(0)(0) === false) @@ -546,30 +553,30 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, _]] .getOrElse("key2", 0) === 2) - val result2 = sql("SELECT data1[key1] FROM mapTable").collect() + val result2 = sql("""SELECT data1["key1"] FROM mapTable""").collect() assert(result2(0)(0) === 1) } test("map with struct values") { - val data = TestSQLContext + val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD data.registerAsTable("mapTable") - val result1 = sql("SELECT data2 FROM mapTable").collect() + val result1 = nestedParserSqlContext.sql("SELECT data2 FROM mapTable").collect() assert(result1.size === 1) val entry1 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] - .getOrElse("7", null) + .getOrElse("seven", null) assert(entry1 != null) assert(entry1(0) === 42) assert(entry1(1) === "the answer") val entry2 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] - .getOrElse("8", null) + .getOrElse("eight", null) assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result2 = sql("SELECT data2[7].payload1, data2[7].payload2 FROM mapTable").collect() + val result2 = nestedParserSqlContext.sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() assert(result2.size === 1) assert(result2(0)(0) === 42.toLong) assert(result2(0)(1) === "the answer") @@ -580,15 +587,15 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA // has no effect in this test case val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) - val result = TestSQLContext + val result = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD result.saveAsParquetFile(tmpdir.toString) - TestSQLContext + nestedParserSqlContext .parquetFile(tmpdir.toString) .toSchemaRDD .registerAsTable("tmpcopy") - val tmpdata = sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() + val tmpdata = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() assert(tmpdata.size === 2) assert(tmpdata(0).size === 2) assert(tmpdata(0)(0) === "Julien Le Dem") @@ -599,34 +606,34 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Writing out Map and reading it back in") { - val data = TestSQLContext + val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) data.saveAsParquetFile(tmpdir.toString) - TestSQLContext + nestedParserSqlContext .parquetFile(tmpdir.toString) .toSchemaRDD .registerAsTable("tmpmapcopy") - val result1 = sql("SELECT data1[key2] FROM tmpmapcopy").collect() + val result1 = nestedParserSqlContext.sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() assert(result1.size === 1) assert(result1(0)(0) === 2) - val result2 = sql("SELECT data2 FROM tmpmapcopy").collect() + val result2 = nestedParserSqlContext.sql("SELECT data2 FROM tmpmapcopy").collect() assert(result2.size === 1) val entry1 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] - .getOrElse("7", null) + .getOrElse("seven", null) assert(entry1 != null) assert(entry1(0) === 42) assert(entry1(1) === "the answer") val entry2 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] - .getOrElse("8", null) + .getOrElse("eight", null) assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result3 = sql("SELECT data2[7].payload1, data2[7].payload2 FROM tmpmapcopy").collect() + val result3 = nestedParserSqlContext.sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() assert(result3.size === 1) assert(result3(0)(0) === 42.toLong) assert(result3(0)(1) === "the answer") @@ -774,3 +781,35 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(mapResult2(2) === 1.3f) } } + +// TODO: the code below is needed temporarily until the standard parser is able to parse +// nested field expressions correctly +class NestedParserSQLContext(@transient override val sparkContext: SparkContext) extends SQLContext(sparkContext) { + override protected[sql] val parser = new NestedSqlParser() +} + +class NestedSqlLexical(override val keywords: Seq[String]) extends SqlLexical(keywords) { + override def identChar = letter | elem('_') + delimiters += (".") +} + +class NestedSqlParser extends SqlParser { + override val lexical = new NestedSqlLexical(reservedWords) + + override protected lazy val baseExpression: PackratParser[Expression] = + expression ~ "[" ~ expression <~ "]" ^^ { + case base ~ _ ~ ordinal => GetItem(base, ordinal) + } | + expression ~ "." ~ ident ^^ { + case base ~ _ ~ fieldName => GetField(base, fieldName) + } | + TRUE ^^^ Literal(true, BooleanType) | + FALSE ^^^ Literal(false, BooleanType) | + cast | + "(" ~> expression <~ ")" | + function | + "-" ~> literal ^^ UnaryMinus | + ident ^^ UnresolvedAttribute | + "*" ^^^ Star(None) | + literal +} From 2f5a805fc9758b395f03b61b8d0142fe54b37ebb Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 24 May 2014 11:57:42 +0300 Subject: [PATCH 32/48] Removing stripMargin from test schemas --- .../spark/sql/parquet/ParquetTestData.scala | 178 +++++++++--------- 1 file changed, 89 insertions(+), 89 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index a11e19f3b6e63..4a07ca567b914 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -56,13 +56,13 @@ private[sql] object ParquetTestData { val testSchema = """message myrecord { - |optional boolean myboolean; - |optional int32 myint; - |optional binary mystring; - |optional int64 mylong; - |optional float myfloat; - |optional double mydouble; - |}""".stripMargin + optional boolean myboolean; + optional int32 myint; + optional binary mystring; + optional int64 mylong; + optional float myfloat; + optional double mydouble; + }""" // field names for test assertion error messages val testSchemaFieldNames = Seq( @@ -76,23 +76,23 @@ private[sql] object ParquetTestData { val subTestSchema = """ - |message myrecord { - |optional boolean myboolean; - |optional int64 mylong; - |} - """.stripMargin + message myrecord { + optional boolean myboolean; + optional int64 mylong; + } + """ val testFilterSchema = """ - |message myrecord { - |required boolean myboolean; - |required int32 myint; - |required binary mystring; - |required int64 mylong; - |required float myfloat; - |required double mydouble; - |} - """.stripMargin + message myrecord { + required boolean myboolean; + required int32 myint; + required binary mystring; + required int64 mylong; + required float myfloat; + required double mydouble; + } + """ // field names for test assertion error messages val subTestSchemaFieldNames = Seq( @@ -118,84 +118,84 @@ private[sql] object ParquetTestData { // The "values" inside ownerPhoneNumbers is a keyword currently // so that array types can be translated correctly. """ - |message AddressBook { - |required binary owner; - |optional group ownerPhoneNumbers { - |repeated binary array; - |} - |optional group contacts { - |repeated group array { - |required binary name; - |optional binary phoneNumber; - |} - |} - |} - """.stripMargin + message AddressBook { + required binary owner; + optional group ownerPhoneNumbers { + repeated binary array; + } + optional group contacts { + repeated group array { + required binary name; + optional binary phoneNumber; + } + } + } + """ val testNestedSchema2 = """ - |message TestNested2 { - |required int32 firstInt; - |optional int32 secondInt; - |optional group longs { - |repeated int64 array; - |} - |required group entries { - |repeated group array { - |required double value; - |optional boolean truth; - |} - |} - |optional group outerouter { - |repeated group array { - |repeated group array { - |repeated int32 array; - |} - |} - |} - |} - """.stripMargin + message TestNested2 { + required int32 firstInt; + optional int32 secondInt; + optional group longs { + repeated int64 array; + } + required group entries { + repeated group array { + required double value; + optional boolean truth; + } + } + optional group outerouter { + repeated group array { + repeated group array { + repeated int32 array; + } + } + } + } + """ val testNestedSchema3 = """ - |message TestNested3 { - |required int32 x; - |optional group booleanNumberPairs { - |repeated group array { - |required int32 key; - |optional group value { - |repeated group array { - |required double nestedValue; - |optional boolean truth; - |} - |} - |} - |} - |} - """.stripMargin + message TestNested3 { + required int32 x; + optional group booleanNumberPairs { + repeated group array { + required int32 key; + optional group value { + repeated group array { + required double nestedValue; + optional boolean truth; + } + } + } + } + } + """ val testNestedSchema4 = """ - |message TestNested4 { - |required int32 x; - |optional group data1 { - |repeated group map { - |required binary key; - |required int32 value; - |} - |} - |required group data2 { - |repeated group map { - |required binary key; - |required group value { - |required int64 payload1; - |optional binary payload2; - |} - |} - |} - |} - """.stripMargin + message TestNested4 { + required int32 x; + optional group data1 { + repeated group map { + required binary key; + required int32 value; + } + } + required group data2 { + repeated group map { + required binary key; + required group value { + required int64 payload1; + optional binary payload2; + } + } + } + } + """ val testNestedDir1 = Utils.createTempDir() val testNestedDir2 = Utils.createTempDir() From de025381ff2d7713e0dc1b2cd096f1cabcc7ba43 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 24 May 2014 12:04:06 +0300 Subject: [PATCH 33/48] Cleaning up ParquetTestData --- .../spark/sql/parquet/ParquetTestData.scala | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 4a07ca567b914..f3677f39cdf64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -205,17 +205,10 @@ private[sql] object ParquetTestData { lazy val testNestedData1 = new ParquetRelation(testNestedDir1.toURI.toString) lazy val testNestedData2 = new ParquetRelation(testNestedDir2.toURI.toString) - // Implicit - // TODO: get rid of this since it is confusing! - implicit def makePath(dir: File): Path = { - new Path(new Path(dir.toURI), new Path("part-r-0.parquet")) - } - def writeFile() = { testDir.delete() val path: Path = new Path(new Path(testDir.toURI), new Path("part-r-0.parquet")) val job = new Job() - val configuration: Configuration = ContextUtil.getConfiguration(job) val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) val writeSupport = new TestGroupWriteSupport(schema) val writer = new ParquetWriter[Group](path, writeSupport) @@ -267,7 +260,7 @@ private[sql] object ParquetTestData { def writeNestedFile1() { // example data from https://blog.twitter.com/2013/dremel-made-simple-with-parquet testNestedDir1.delete() - val path: Path = testNestedDir1 + val path: Path = new Path(new Path(testNestedDir1.toURI), new Path("part-r-0.parquet")) val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema1) val r1 = new SimpleGroup(schema) @@ -295,7 +288,7 @@ private[sql] object ParquetTestData { def writeNestedFile2() { testNestedDir2.delete() - val path: Path = testNestedDir2 + val path: Path = new Path(new Path(testNestedDir2.toURI), new Path("part-r-0.parquet")) val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema2) val r1 = new SimpleGroup(schema) @@ -332,7 +325,7 @@ private[sql] object ParquetTestData { def writeNestedFile3() { testNestedDir3.delete() - val path: Path = testNestedDir3 + val path: Path = new Path(new Path(testNestedDir3.toURI), new Path("part-r-0.parquet")) val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema3) val r1 = new SimpleGroup(schema) @@ -362,7 +355,7 @@ private[sql] object ParquetTestData { def writeNestedFile4() { testNestedDir4.delete() - val path: Path = testNestedDir4 + val path: Path = new Path(new Path(testNestedDir4.toURI), new Path("part-r-0.parquet")) val schema: MessageType = MessageTypeParser.parseMessageType(testNestedSchema4) val r1 = new SimpleGroup(schema) @@ -393,8 +386,10 @@ private[sql] object ParquetTestData { writer.close() } - def readNestedFile(path: File, schemaString: String): Unit = { + /* // TODO: this is not actually used anywhere but useful for debugging + def readNestedFile(file: File, schemaString: String): Unit = { val configuration = new Configuration() + val path = new Path(new Path(file.toURI), new Path("part-r-0.parquet")) val fs: FileSystem = path.getFileSystem(configuration) val schema: MessageType = MessageTypeParser.parseMessageType(schemaString) assert(schema != null) @@ -404,6 +399,6 @@ private[sql] object ParquetTestData { val reader = new ParquetReader(new Path(path.toString), new GroupReadSupport()) val first = reader.read() assert(first != null) - } + }*/ } From 31465d6d8e47c61f6cc32935c9befb754badff64 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sat, 24 May 2014 12:11:49 +0300 Subject: [PATCH 34/48] Scalastyle: fixing commented out bottom --- .../org/apache/spark/sql/parquet/ParquetTestData.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index f3677f39cdf64..1dc58633a2a68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -386,8 +386,8 @@ private[sql] object ParquetTestData { writer.close() } - /* // TODO: this is not actually used anywhere but useful for debugging - def readNestedFile(file: File, schemaString: String): Unit = { + // TODO: this is not actually used anywhere but useful for debugging + /* def readNestedFile(file: File, schemaString: String): Unit = { val configuration = new Configuration() val path = new Path(new Path(file.toURI), new Path("part-r-0.parquet")) val fs: FileSystem = path.getFileSystem(configuration) @@ -399,6 +399,6 @@ private[sql] object ParquetTestData { val reader = new ParquetReader(new Path(path.toString), new GroupReadSupport()) val first = reader.read() assert(first != null) - }*/ + } */ } From 3c6b25ffa56e45a366ec8aac928f8fd57b785098 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 1 Jun 2014 15:43:37 +0300 Subject: [PATCH 35/48] Trying to reduce no-op changes wrt master --- .../spark/sql/catalyst/expressions/Row.scala | 1 + .../catalyst/plans/logical/LogicalPlan.scala | 20 ++----------------- 2 files changed, 3 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 71825eca73b80..74ae723686cfe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -206,6 +206,7 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { override def copy() = new GenericRow(values.clone()) } + class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = this(ordering.map(BindReferences.bindReference(_, inputSchema))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 76459f49cae02..0933a31c362d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.catalyst.types._ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => @@ -54,11 +54,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { /** * Optionally resolves the given string to a * [[catalyst.expressions.NamedExpression NamedExpression]]. The attribute is expressed as - * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. Fields - * can contain ordinal expressions, such as `field[i][j][k]...`. + * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ def resolve(name: String): Option[NamedExpression] = { - // TODO: extend SqlParser to handle field expressions val parts = name.split("\\.") // Collect all attributes that are output by this nodes children where either the first part // matches the name or where the first part matches the scope and the second part matches the @@ -69,20 +67,6 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { val remainingParts = if (option.qualifiers.contains(parts.head) && parts.size > 1) parts.drop(1) else parts if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil - // TODO from rebase! - /*val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts - val relevantRemaining = - if (remainingParts.head.matches("\\w*\\[(\\d+|\\w+)\\]")) { // array field name - remainingParts.head.substring(0, remainingParts.head.indexOf("[")) - } else { - remainingParts.head - } - if (option.name == relevantRemaining) (option, remainingParts.tail.toList) :: Nil else Nil*/ - // If the first part of the desired name matches a qualifier for this possible match, drop it. - /* TODO: from rebase! - val remainingParts = if (option.qualifiers contains parts.head) parts.drop(1) else parts - if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil - */ } options.distinct match { From 3104886615f8a79de38b4504b5a218881740f5a1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 3 Jun 2014 13:16:38 -0700 Subject: [PATCH 36/48] Nested Rows should be Rows, not Seqs. --- .../apache/spark/sql/catalyst/expressions/complexTypes.scala | 2 +- .../scala/org/apache/spark/sql/parquet/ParquetConverter.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 37ccb965feb87..5d3bb25ad568c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -94,7 +94,7 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] override def eval(input: Row): Any = { - val baseValue = child.eval(input).asInstanceOf[Seq[_]] + val baseValue = child.eval(input).asInstanceOf[Row] if (baseValue == null) null else baseValue(ordinal) } 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 27c4c2ac76487..53a8575349806 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 @@ -564,7 +564,7 @@ private[parquet] class CatalystStructConverter( // 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, current.toArray.toSeq) + parent.updateField(index, new GenericRow(current.toArray)) } } From f7aeba3427dded478f3dc0fa11eb0647acaec9db Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 3 Jun 2014 13:17:07 -0700 Subject: [PATCH 37/48] [SPARK-1982] Support for ByteType and ShortType. --- .../main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 5843f68ed41d0..d5516fc7bac2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -170,6 +170,9 @@ private[parquet] object ParquetTypesConverter { Some(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) case FloatType => Some(ParquetPrimitiveTypeName.FLOAT) case IntegerType => Some(ParquetPrimitiveTypeName.INT32) + // There is no type for Byte or Short so we promote them to INT32. + case ShortType => Some(ParquetPrimitiveTypeName.INT32) + case ByteType => Some(ParquetPrimitiveTypeName.INT32) case LongType => Some(ParquetPrimitiveTypeName.INT64) case _ => None } From 3e1456c961c2631975da0119ba8e0d9c50a2b814 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 4 Jun 2014 01:31:08 -0700 Subject: [PATCH 38/48] WIP: Directly serialize catalyst attributes. --- .../spark/sql/catalyst/types/dataTypes.scala | 79 ++++++++++++++++++- .../sql/parquet/ParquetTableOperations.scala | 5 +- .../sql/parquet/ParquetTableSupport.scala | 26 +++--- .../spark/sql/parquet/ParquetQuerySuite.scala | 19 ++++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +- 5 files changed, 110 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index a3484cc3de870..b8030611760c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -19,12 +19,72 @@ package org.apache.spark.sql.catalyst.types import java.sql.Timestamp +import scala.util.parsing.combinator.RegexParsers + import scala.reflect.ClassTag import scala.reflect.runtime.universe.{typeTag, TypeTag, runtimeMirror} -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.util.Utils +/** + * + */ +object DataType extends RegexParsers { + protected lazy val primitiveType: Parser[DataType] = + "StringType" ^^^ StringType | + "FloatType" ^^^ FloatType | + "IntegerType" ^^^ IntegerType | + "ByteType" ^^^ ByteType | + "ShortType" ^^^ ShortType | + "DoubleType" ^^^ DoubleType | + "LongType" ^^^ LongType | + "BinaryType" ^^^ BinaryType | + "BooleanType" ^^^ BooleanType | + "DecimalType" ^^^ DecimalType | + "TimestampType" ^^^ TimestampType + + protected lazy val arrayType: Parser[DataType] = + "ArrayType" ~> "(" ~> dataType <~ ")" ^^ ArrayType + + protected lazy val mapType: Parser[DataType] = + "MapType" ~> "(" ~> dataType ~ "," ~ dataType <~ ")" ^^ { + case t1 ~ _ ~ t2 => MapType(t1, t2) + } + + protected lazy val structField: Parser[StructField] = + ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { + case name ~ tpe ~ nullable => + StructField(name, tpe, nullable = nullable) + } + + protected lazy val boolVal: Parser[Boolean] = + "true" ^^^ true | + "false" ^^^ false + + + protected lazy val structType: Parser[DataType] = + "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { + case fields => new StructType(fields) + } + + protected lazy val dataType: Parser[DataType] = + arrayType | + mapType | + structType | + primitiveType + + /** + * Parses a string representation of a DataType. + * + * TODO: Generate parser as pickler... + */ + def apply(asString: String): DataType = parseAll(dataType, asString) match { + case Success(result, _) => result + case failure: NoSuccess => sys.error(s"Unsupported dataType: $asString, $failure") + } +} + abstract class DataType { /** Matches any expression that evaluates to this DataType */ def unapply(a: Expression): Boolean = a match { @@ -32,13 +92,13 @@ abstract class DataType { case _ => false } - def isPrimitive(): Boolean = false + def isPrimitive: Boolean = false } case object NullType extends DataType trait PrimitiveType extends DataType { - override def isPrimitive() = true + override def isPrimitive = true } abstract class NativeType extends DataType { @@ -167,6 +227,17 @@ case object FloatType extends FractionalType { case class ArrayType(elementType: DataType) extends DataType case class StructField(name: String, dataType: DataType, nullable: Boolean) -case class StructType(fields: Seq[StructField]) extends DataType + +object StructType { + def fromAttributes(attributes: Seq[Attribute]): StructType = { + StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) + } + + //def apply(fields: Seq[StructField]) = new StructType(fields.toIndexedSeq) +} + +case class StructType(fields: Seq[StructField]) extends DataType { + def toAttributes = fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)()) +} case class MapType(keyType: DataType, valueType: DataType) extends DataType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index b11888af45853..97c56710d2b0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -36,6 +36,7 @@ import parquet.schema.MessageType import org.apache.spark.{Logging, SerializableWritable, SparkContext, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} +import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} /** @@ -167,7 +168,7 @@ case class InsertIntoParquetTable( val job = new Job(sc.hadoopConfiguration) val writeSupport = - if (child.output.map(_.dataType).forall(_.isPrimitive())) { + if (child.output.map(_.dataType).forall(_.isPrimitive)) { logger.debug("Initializing MutableRowWriteSupport") classOf[org.apache.spark.sql.parquet.MutableRowWriteSupport] } else { @@ -178,7 +179,7 @@ case class InsertIntoParquetTable( // TODO: move that to function in object val conf = ContextUtil.getConfiguration(job) - conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString) + conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, StructType.fromAttributes(relation.output).toString) val fspath = new Path(relation.path) val fs = fspath.getFileSystem(conf) 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 f839fdc961df4..b2e9a8d5554f9 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 @@ -82,39 +82,35 @@ private[parquet] object RowReadSupport { * A `parquet.hadoop.api.WriteSupport` for Row ojects. */ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { - def setSchema(schema: MessageType, configuration: Configuration) { - // for testing - this.schema = schema - // TODO: could use Attributes themselves instead of Parquet schema? + + + def setSchema(schema: Seq[Attribute], configuration: Configuration) { configuration.set( RowWriteSupport.PARQUET_ROW_SCHEMA, - schema.toString) + StructType.fromAttributes(schema).toString) configuration.set( ParquetOutputFormat.WRITER_VERSION, ParquetProperties.WriterVersion.PARQUET_1_0.toString) } - def getSchema(configuration: Configuration): MessageType = { - MessageTypeParser.parseMessageType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) - } - - private[parquet] var schema: MessageType = null private[parquet] var writer: RecordConsumer = null private[parquet] var attributes: Seq[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { - schema = if (schema == null) getSchema(configuration) else schema - attributes = ParquetTypesConverter.convertToAttributes(schema) - log.debug(s"write support initialized for requested schema $schema") + attributes = DataType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) match { + case s: StructType => s.toAttributes + case other => sys.error(s"Can convert $attributes to row") + } + log.debug(s"write support initialized for requested schema $attributes") ParquetRelation.enableLogForwarding() new WriteSupport.WriteContext( - schema, + ParquetTypesConverter.convertFromAttributes(attributes), new java.util.HashMap[java.lang.String, java.lang.String]()) } override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { writer = recordConsumer - log.debug(s"preparing for write with schema $schema") + log.debug(s"preparing for write with schema $attributes") } override def write(record: Row): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 3cf7b0f10d09e..950f729dba0ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -67,7 +67,17 @@ case class Nested(i: Int, s: String) case class Data(array: Seq[Int], nested: Nested) -class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { +case class AllDataTypes( + stringField: String, + intField: Int, + longField: Long, + floatField: Float, + doubleField: Double, + shortField: Short, + byteField: Byte, + booleanField: Boolean) + +class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { import TestData._ TestData // Load test data tables. @@ -100,6 +110,13 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA // here we should also unregister the table?? } + test("Read/Write All Types") { + val data = AllDataTypes("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true) + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + sparkContext.parallelize(data :: Nil).saveAsParquetFile(tempDir) + assert(parquetFile(tempDir).collect().head === data) + } + test("self-join parquet files") { val x = ParquetTestData.testData.as('x) val y = ParquetTestData.testData.as('y) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 68284344afd55..f923d68932f83 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -208,7 +208,9 @@ object HiveMetastoreTypes extends RegexParsers { } protected lazy val structType: Parser[DataType] = - "struct" ~> "<" ~> repsep(structField,",") <~ ">" ^^ StructType + "struct" ~> "<" ~> repsep(structField,",") <~ ">" ^^ { + case fields => new StructType(fields) + } protected lazy val dataType: Parser[DataType] = arrayType | From 14c3fd8a3c39010e984b91010a7959c2e17e0d68 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 4 Jun 2014 21:16:49 +0300 Subject: [PATCH 39/48] Attempting to fix Spark-Parquet schema conversion --- .../spark/sql/parquet/ParquetConverter.scala | 4 +- .../sql/parquet/ParquetTableOperations.scala | 11 ++- .../sql/parquet/ParquetTableSupport.scala | 88 ++++++++++++++++--- .../spark/sql/parquet/ParquetTypes.scala | 13 +++ 4 files changed, 97 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 53a8575349806..656ba56d4f86e 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 @@ -106,8 +106,8 @@ private[sql] object CatalystConverter { } } - protected[parquet] def createRootConverter(parquetSchema: MessageType): CatalystConverter = { - val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) + protected[parquet] def createRootConverter(parquetSchema: MessageType, attributes: Seq[Attribute]): CatalystConverter = { + //val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) // For non-nested types we use the optimized Row converter if (attributes.forall(a => ParquetTypesConverter.isPrimitiveType(a.dataType))) { new CatalystPrimitiveRowConverter(attributes) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 97c56710d2b0c..5b401f1003631 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -66,9 +66,13 @@ case class ParquetTableScan( } // Store Parquet schema in `Configuration` + // TODO: should this here be just the projected fields? conf.set( - RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertFromAttributes(output).toString) + RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, + ParquetTypesConverter.convertToString(output)) + //conf.set( + // RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, + // ParquetTypesConverter.convertFromAttributes(output).toString) // Store record filtering predicate in `Configuration` // Note 1: the input format ignores all predicates that cannot be expressed @@ -179,7 +183,8 @@ case class InsertIntoParquetTable( // TODO: move that to function in object val conf = ContextUtil.getConfiguration(job) - conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, StructType.fromAttributes(relation.output).toString) + //conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, StructType.fromAttributes(relation.output).toString) + RowWriteSupport.setSchema(relation.output, conf) val fspath = new Path(relation.path) val fs = fspath.getFileSystem(conf) 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 b2e9a8d5554f9..a569f9802ccf8 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 @@ -29,6 +29,8 @@ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.SparkSqlSerializer +import com.google.common.io.BaseEncoding /** * A `parquet.io.api.RecordMaterializer` for Rows. @@ -38,8 +40,8 @@ import org.apache.spark.sql.catalyst.types._ private[parquet] class RowRecordMaterializer(root: CatalystConverter) extends RecordMaterializer[Row] { - def this(parquetSchema: MessageType) = - this(CatalystConverter.createRootConverter(parquetSchema)) + def this(parquetSchema: MessageType, attributes: Seq[Attribute]) = + this(CatalystConverter.createRootConverter(parquetSchema, attributes)) override def getCurrentRecord: Row = root.getCurrentRecord @@ -57,25 +59,62 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { fileSchema: MessageType, readContext: ReadContext): RecordMaterializer[Row] = { log.debug(s"preparing for read with file schema $fileSchema") - new RowRecordMaterializer(readContext.getRequestedSchema) + //new RowRecordMaterializer(readContext.getRequestedSchema) + val parquetSchema = readContext.getRequestedSchema + var schema: Seq[Attribute] = + if (readContext.getReadSupportMetadata != null && + readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_METADATA_KEY) != null) { + ParquetTypesConverter.convertFromString( + readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) + } else { + // fall back to converting from Parquet schema + ParquetTypesConverter.convertToAttributes(parquetSchema) + } + new RowRecordMaterializer(parquetSchema, schema) } override def init( configuration: Configuration, keyValueMetaData: java.util.Map[String, String], fileSchema: MessageType): ReadContext = { - val requested_schema_string = + /*val requested_schema_string = configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString) val requested_schema = MessageTypeParser.parseMessageType(requested_schema_string) log.debug(s"read support initialized for requested schema $requested_schema") ParquetRelation.enableLogForwarding() - new ReadContext(requested_schema, keyValueMetaData) + new ReadContext(requested_schema, keyValueMetaData) */ + + // GO ON HERE.. figure out why Avro distinguishes between requested read and read schema + // try to figure out what when needs to be written to metadata + + var parquetSchema: MessageType = fileSchema + var metadata: java.util.Map[String, String] = null + val requestedAttributes = RowReadSupport.getRequestedSchema(configuration) + + if (requestedAttributes != null) { + parquetSchema = ParquetTypesConverter.convertFromAttributes(requestedAttributes) + } + + val origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) + + if (origAttributesStr != null) { + metadata = new java.util.HashMap[String, String]() + metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) + } + + return new ReadSupport.ReadContext(parquetSchema, metadata) } } private[parquet] object RowReadSupport { - val PARQUET_ROW_REQUESTED_SCHEMA = "org.apache.spark.sql.parquet.row.requested_schema" + val SPARK_ROW_REQUESTED_SCHEMA = "org.apache.spark.sql.parquet.row.requested_schema" + val SPARK_METADATA_KEY = "org.apache.spark.sql.parquet.row.metadata" + + private def getRequestedSchema(configuration: Configuration): Seq[Attribute] = { + val schemaString = configuration.get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA) + if (schemaString == null) null else ParquetTypesConverter.convertFromString(schemaString) + } } /** @@ -83,24 +122,29 @@ private[parquet] object RowReadSupport { */ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { - - def setSchema(schema: Seq[Attribute], configuration: Configuration) { + /*def setSchema(schema: Seq[Attribute], configuration: Configuration) { configuration.set( RowWriteSupport.PARQUET_ROW_SCHEMA, StructType.fromAttributes(schema).toString) configuration.set( ParquetOutputFormat.WRITER_VERSION, ParquetProperties.WriterVersion.PARQUET_1_0.toString) - } + } */ + private[parquet] var schema: MessageType = null private[parquet] var writer: RecordConsumer = null private[parquet] var attributes: Seq[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { - attributes = DataType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) match { - case s: StructType => s.toAttributes - case other => sys.error(s"Can convert $attributes to row") - } + //attributes = DataType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) + attributes = if (attributes == null) { + RowWriteSupport.getSchema(configuration) match { + case s: StructType => s.toAttributes + case other => sys.error(s"Can convert $attributes to row") + } + } else attributes + schema = if (schema == null) ParquetTypesConverter.convertFromAttributes(attributes) else schema + // ParquetTypesConverter.convertToAttributes(schema) log.debug(s"write support initialized for requested schema $attributes") ParquetRelation.enableLogForwarding() new WriteSupport.WriteContext( @@ -275,6 +319,22 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { } private[parquet] object RowWriteSupport { - val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema" + val SPARK_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.attributes" + + def getSchema(configuration: Configuration): Seq[Attribute] = { + val schemaString = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) + if (schemaString == null) { + throw new RuntimeException("Missing schema!") + } + ParquetTypesConverter.convertFromString(schemaString) + } + + def setSchema(schema: Seq[Attribute], configuration: Configuration) { + val encoded = ParquetTypesConverter.convertToString(schema) + configuration.set(SPARK_ROW_SCHEMA, encoded) + configuration.set( + ParquetOutputFormat.WRITER_VERSION, + ParquetProperties.WriterVersion.PARQUET_1_0.toString) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index d5516fc7bac2d..17df1fc410720 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -33,6 +33,8 @@ import parquet.schema.Type.Repetition import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} import org.apache.spark.sql.catalyst.types._ +import com.google.common.io.BaseEncoding +import org.apache.spark.sql.execution.SparkSqlSerializer // Implicits import scala.collection.JavaConversions._ @@ -289,6 +291,16 @@ private[parquet] object ParquetTypesConverter { new MessageType("root", fields) } + def convertFromString(string: String): Seq[Attribute] = { + val decoded: Array[Byte] = BaseEncoding.base64().decode(string) + SparkSqlSerializer.deserialize(decoded) + } + + def convertToString(schema: Seq[Attribute]): String = { + val serialized: Array[Byte] = SparkSqlSerializer.serialize(schema) + BaseEncoding.base64().encode(serialized) + } + def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { if (origPath == null) { throw new IllegalArgumentException("Unable to write Parquet metadata: path is null") @@ -313,6 +325,7 @@ private[parquet] object ParquetTypesConverter { } val extraMetadata = new java.util.HashMap[String, String]() extraMetadata.put("path", path.toString) + extraMetadata.put(RowReadSupport.SPARK_METADATA_KEY, ParquetTypesConverter.convertToString(attributes)) // TODO: add extra data, e.g., table name, date, etc.? val parquetSchema: MessageType = From 37e0a0a7e6520453ec77b6e2278318a352cd8ea5 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 4 Jun 2014 21:24:42 +0300 Subject: [PATCH 40/48] Cleaning up --- .../apache/spark/sql/parquet/ParquetConverter.scala | 5 +++-- .../spark/sql/parquet/ParquetTableOperations.scala | 5 ----- .../spark/sql/parquet/ParquetTableSupport.scala | 13 +------------ .../org/apache/spark/sql/parquet/ParquetTypes.scala | 4 +++- 4 files changed, 7 insertions(+), 20 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 656ba56d4f86e..9b702d7a99f20 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 @@ -106,8 +106,9 @@ private[sql] object CatalystConverter { } } - protected[parquet] def createRootConverter(parquetSchema: MessageType, attributes: Seq[Attribute]): CatalystConverter = { - //val attributes = ParquetTypesConverter.convertToAttributes(parquetSchema) + 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) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 5b401f1003631..69817255303e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -66,13 +66,9 @@ case class ParquetTableScan( } // Store Parquet schema in `Configuration` - // TODO: should this here be just the projected fields? conf.set( RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertToString(output)) - //conf.set( - // RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, - // ParquetTypesConverter.convertFromAttributes(output).toString) // Store record filtering predicate in `Configuration` // Note 1: the input format ignores all predicates that cannot be expressed @@ -181,7 +177,6 @@ case class InsertIntoParquetTable( ParquetOutputFormat.setWriteSupportClass(job, writeSupport) - // TODO: move that to function in object val conf = ContextUtil.getConfiguration(job) //conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, StructType.fromAttributes(relation.output).toString) RowWriteSupport.setSchema(relation.output, conf) 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 a569f9802ccf8..4212fc9a0a763 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 @@ -59,7 +59,7 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { fileSchema: MessageType, readContext: ReadContext): RecordMaterializer[Row] = { log.debug(s"preparing for read with file schema $fileSchema") - //new RowRecordMaterializer(readContext.getRequestedSchema) + // Note: this very much imitates AvroParquet val parquetSchema = readContext.getRequestedSchema var schema: Seq[Attribute] = if (readContext.getReadSupportMetadata != null && @@ -77,17 +77,6 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { configuration: Configuration, keyValueMetaData: java.util.Map[String, String], fileSchema: MessageType): ReadContext = { - /*val requested_schema_string = - configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString) - val requested_schema = - MessageTypeParser.parseMessageType(requested_schema_string) - log.debug(s"read support initialized for requested schema $requested_schema") - ParquetRelation.enableLogForwarding() - new ReadContext(requested_schema, keyValueMetaData) */ - - // GO ON HERE.. figure out why Avro distinguishes between requested read and read schema - // try to figure out what when needs to be written to metadata - var parquetSchema: MessageType = fileSchema var metadata: java.util.Map[String, String] = null val requestedAttributes = RowReadSupport.getRequestedSchema(configuration) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 17df1fc410720..e67f35e8d03b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -325,7 +325,9 @@ private[parquet] object ParquetTypesConverter { } val extraMetadata = new java.util.HashMap[String, String]() extraMetadata.put("path", path.toString) - extraMetadata.put(RowReadSupport.SPARK_METADATA_KEY, ParquetTypesConverter.convertToString(attributes)) + extraMetadata.put( + RowReadSupport.SPARK_METADATA_KEY, + ParquetTypesConverter.convertToString(attributes)) // TODO: add extra data, e.g., table name, date, etc.? val parquetSchema: MessageType = From 88e6bdb5f8d41264d2b62f6af8b562d97539c4cc Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 4 Jun 2014 22:28:56 +0300 Subject: [PATCH 41/48] Attempting to fix loss of schema --- .../apache/spark/sql/parquet/ParquetRelation.scala | 6 +++++- .../spark/sql/parquet/ParquetTableSupport.scala | 11 +++++------ .../org/apache/spark/sql/parquet/ParquetTypes.scala | 9 +++++---- 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index e581fb0ae71c9..48089e06ca117 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -55,6 +55,8 @@ private[sql] case class ParquetRelation(val path: String) .getSchema /** Attributes */ + // TODO: THIS POTENTIALLY LOOSES TYPE INFORMATION!!!! + // e.g. short <-> INT32 and byte <-> INT32 override val output = ParquetTypesConverter .convertToAttributes(parquetSchema) @@ -132,7 +134,9 @@ private[sql] object ParquetRelation { } ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) - new ParquetRelation(path.toString) + new ParquetRelation(path.toString) { + override val output = attributes + } } private def checkPath(pathStr: String, allowExisting: Boolean, conf: Configuration): Path = { 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 4212fc9a0a763..fbe0075688e0c 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 @@ -126,12 +126,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { override def init(configuration: Configuration): WriteSupport.WriteContext = { //attributes = DataType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) - attributes = if (attributes == null) { - RowWriteSupport.getSchema(configuration) match { - case s: StructType => s.toAttributes - case other => sys.error(s"Can convert $attributes to row") - } - } else attributes + attributes = if (attributes == null) RowWriteSupport.getSchema(configuration) else attributes schema = if (schema == null) ParquetTypesConverter.convertFromAttributes(attributes) else schema // ParquetTypesConverter.convertToAttributes(schema) log.debug(s"write support initialized for requested schema $attributes") @@ -192,7 +187,9 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { ) ) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) + case ShortType => writer.addInteger(value.asInstanceOf[Int]) case LongType => writer.addLong(value.asInstanceOf[Long]) + case ByteType => writer.addInteger(value.asInstanceOf[Int]) case DoubleType => writer.addDouble(value.asInstanceOf[Double]) case FloatType => writer.addFloat(value.asInstanceOf[Float]) case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) @@ -298,7 +295,9 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { ) ) case IntegerType => writer.addInteger(record.getInt(index)) + case ShortType => writer.addInteger(record.getShort(index)) case LongType => writer.addLong(record.getLong(index)) + case ByteType => writer.addInteger(record.getByte(index)) case DoubleType => writer.addDouble(record.getDouble(index)) case FloatType => writer.addFloat(record.getFloat(index)) case BooleanType => writer.addBoolean(record.getBoolean(index)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index e67f35e8d03b3..35eb622a4310a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -292,13 +292,14 @@ private[parquet] object ParquetTypesConverter { } def convertFromString(string: String): Seq[Attribute] = { - val decoded: Array[Byte] = BaseEncoding.base64().decode(string) - SparkSqlSerializer.deserialize(decoded) + DataType(string) match { + case s: StructType => s.toAttributes + case other => sys.error(s"Can convert $string to row") + } } def convertToString(schema: Seq[Attribute]): String = { - val serialized: Array[Byte] = SparkSqlSerializer.serialize(schema) - BaseEncoding.base64().encode(serialized) + StructType.fromAttributes(schema).toString } def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { From 63d1b5710f26a848b31f1fe0d20fe636cd115f20 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 8 Jun 2014 12:29:21 +0300 Subject: [PATCH 42/48] Cleaning up and Scalastyle --- .../spark/sql/catalyst/types/dataTypes.scala | 2 +- .../spark/sql/parquet/ParquetTableOperations.scala | 1 - .../spark/sql/parquet/ParquetTableSupport.scala | 14 +------------- 3 files changed, 2 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index b8030611760c6..bb77bccf86176 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -233,7 +233,7 @@ object StructType { StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) } - //def apply(fields: Seq[StructField]) = new StructType(fields.toIndexedSeq) + // def apply(fields: Seq[StructField]) = new StructType(fields.toIndexedSeq) } case class StructType(fields: Seq[StructField]) extends DataType { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 69817255303e7..e52431d7c53e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -178,7 +178,6 @@ case class InsertIntoParquetTable( ParquetOutputFormat.setWriteSupportClass(job, writeSupport) val conf = ContextUtil.getConfiguration(job) - //conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, StructType.fromAttributes(relation.output).toString) RowWriteSupport.setSchema(relation.output, conf) val fspath = new Path(relation.path) 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 fbe0075688e0c..f4ce210266fcd 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 @@ -111,24 +111,12 @@ private[parquet] object RowReadSupport { */ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { - /*def setSchema(schema: Seq[Attribute], configuration: Configuration) { - configuration.set( - RowWriteSupport.PARQUET_ROW_SCHEMA, - StructType.fromAttributes(schema).toString) - configuration.set( - ParquetOutputFormat.WRITER_VERSION, - ParquetProperties.WriterVersion.PARQUET_1_0.toString) - } */ - - private[parquet] var schema: MessageType = null private[parquet] var writer: RecordConsumer = null private[parquet] var attributes: Seq[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { - //attributes = DataType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) attributes = if (attributes == null) RowWriteSupport.getSchema(configuration) else attributes - schema = if (schema == null) ParquetTypesConverter.convertFromAttributes(attributes) else schema - // ParquetTypesConverter.convertToAttributes(schema) + log.debug(s"write support initialized for requested schema $attributes") ParquetRelation.enableLogForwarding() new WriteSupport.WriteContext( From b8a8b9a28baf2780de7841d269a17f01db41e10b Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 8 Jun 2014 16:57:42 +0300 Subject: [PATCH 43/48] More fixes to short and byte conversion --- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../spark/sql/parquet/ParquetConverter.scala | 67 +++++++++++++------ .../spark/sql/parquet/ParquetRelation.scala | 6 +- .../sql/parquet/ParquetTableOperations.scala | 5 +- .../spark/sql/parquet/ParquetTypes.scala | 41 +++++++++--- .../spark/sql/parquet/ParquetQuerySuite.scala | 19 +++++- 6 files changed, 100 insertions(+), 41 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index feb280d1d1411..4694f25d6d630 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -154,7 +154,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.WriteToFile(path, child) => val relation = ParquetRelation.create(path, child, sparkContext.hadoopConfiguration) - InsertIntoParquetTable(relation, planLater(child), overwrite=true)(sparkContext) :: Nil + // Note: overwrite=false because otherwise the metadata we just created will be deleted + InsertIntoParquetTable(relation, planLater(child), overwrite=false)(sparkContext) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => InsertIntoParquetTable(table, planLater(child), overwrite)(sparkContext) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => 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 9b702d7a99f20..9f72d72cd7437 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 @@ -93,14 +93,30 @@ private[sql] object CatalystConverter { fieldIndex, parent) } - case ctype: NativeType => { - // note: for some reason matching for StringType fails so use this ugly if instead - if (ctype == StringType) { - new CatalystPrimitiveStringConverter(parent, fieldIndex) - } else { - new CatalystPrimitiveConverter(parent, fieldIndex) + // Strings, Shorts and Bytes do not have a corresponding type in Parquet + // so we need to treat them separately + case StringType => { + new CatalystPrimitiveConverter(parent, fieldIndex) { + override def addBinary(value: Binary): Unit = + parent.updateString(fieldIndex, value) } } + case ShortType => { + new CatalystPrimitiveConverter(parent, fieldIndex) { + override def addInt(value: Int): Unit = + parent.updateShort(fieldIndex, value.asInstanceOf[ShortType.JvmType]) + } + } + case ByteType => { + new CatalystPrimitiveConverter(parent, fieldIndex) { + override def addInt(value: Int): Unit = + parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.JvmType]) + } + } + // All other primitive types use the default converter + case ctype: NativeType => { // note: need the type tag here! + new CatalystPrimitiveConverter(parent, fieldIndex) + } case _ => throw new RuntimeException( s"unable to convert datatype ${field.dataType.toString} in CatalystConverter") } @@ -153,6 +169,12 @@ private[parquet] trait CatalystConverter { 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) @@ -309,6 +331,12 @@ private[parquet] class CatalystPrimitiveRowConverter( 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) @@ -350,21 +378,6 @@ private[parquet] class CatalystPrimitiveConverter( parent.updateLong(fieldIndex, value) } -/** - * A `parquet.io.api.PrimitiveConverter` that converts Parquet strings (fixed-length byte arrays) - * into Catalyst Strings. - * - * @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) { - override def addBinary(value: Binary): Unit = - parent.updateString(fieldIndex, value) -} - object CatalystArrayConverter { val INITIAL_ARRAY_SIZE = 20 } @@ -486,6 +499,18 @@ private[parquet] class CatalystNativeArrayConverter( 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] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 48089e06ca117..67edfd8a19025 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -55,11 +55,7 @@ private[sql] case class ParquetRelation(val path: String) .getSchema /** Attributes */ - // TODO: THIS POTENTIALLY LOOSES TYPE INFORMATION!!!! - // e.g. short <-> INT32 and byte <-> INT32 - override val output = - ParquetTypesConverter - .convertToAttributes(parquetSchema) + override val output = ParquetTypesConverter.readSchemaFromFile(new Path(path)) override def newInstance = ParquetRelation(path).asInstanceOf[this.type] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index e52431d7c53e2..624f2e2fa13f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -65,10 +65,13 @@ case class ParquetTableScan( NewFileInputFormat.addInputPath(job, path) } - // Store Parquet schema in `Configuration` + // Store both requested and original schema in `Configuration` conf.set( RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertToString(output)) + conf.set( + RowWriteSupport.SPARK_ROW_SCHEMA, + ParquetTypesConverter.convertToString(relation.output)) // Store record filtering predicate in `Configuration` // Note 1: the input format ignores all predicates that cannot be expressed diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 35eb622a4310a..925da184f3d3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -26,20 +26,19 @@ import org.apache.hadoop.mapreduce.Job import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter} import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData} import parquet.hadoop.util.ContextUtil -import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser} +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType} import parquet.schema.{GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import parquet.schema.Type.Repetition +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} import org.apache.spark.sql.catalyst.types._ -import com.google.common.io.BaseEncoding -import org.apache.spark.sql.execution.SparkSqlSerializer // Implicits import scala.collection.JavaConversions._ -private[parquet] object ParquetTypesConverter { +private[parquet] object ParquetTypesConverter extends Logging { def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass @@ -62,7 +61,7 @@ private[parquet] object ParquetTypesConverter { * Converts a given Parquet `Type` into the corresponding * [[org.apache.spark.sql.catalyst.types.DataType]]. * - * Note that we apply the following conversion rules: + * We apply the following conversion rules: *
        *
      • Primitive types are converter to the corresponding primitive type.
      • *
      • Group types that have a single field that is itself a group, which has repetition @@ -97,6 +96,7 @@ private[parquet] object ParquetTypesConverter { keyValueGroup.getFields.apply(1).getName == CatalystConverter.MAP_VALUE_SCHEMA_NAME } } + def correspondsToArray(groupType: ParquetGroupType): Boolean = { groupType.getFieldCount == 1 && groupType.getFieldName(0) == CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME && @@ -188,7 +188,7 @@ private[parquet] object ParquetTypesConverter { *
      • Primitive types are converted into Parquet's primitive types.
      • *
      • [[org.apache.spark.sql.catalyst.types.StructType]]s are converted * into Parquet's `GroupType` with the corresponding field types.
      • - *
      • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are converterd + *
      • [[org.apache.spark.sql.catalyst.types.ArrayType]]s are converted * into a 2-level nested group, where the outer group has the inner * group as sole field. The inner group has name `values` and * repetition level `REPEATED` and has the element type of @@ -269,9 +269,6 @@ private[parquet] object ParquetTypesConverter { } } - def getSchema(schemaString: String) : MessageType = - MessageTypeParser.parseMessageType(schemaString) - def convertToAttributes(parquetSchema: ParquetType): Seq[Attribute] = { parquetSchema .asGroupType() @@ -302,7 +299,7 @@ private[parquet] object ParquetTypesConverter { StructType.fromAttributes(schema).toString } - def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration) { + def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration): Unit = { if (origPath == null) { throw new IllegalArgumentException("Unable to write Parquet metadata: path is null") } @@ -385,4 +382,28 @@ private[parquet] object ParquetTypesConverter { footers(0).getParquetMetadata } } + + /** + * Reads in Parquet Metadata from the given path and tries to extract the schema + * (Catalyst attributes) from the application-specific key-value map. If this + * is empty it falls back to converting from the Parquet file schema which + * may lead to an upcast of types (e.g., {byte, short} to int). + * + * @param origPath The path at which we expect one (or more) Parquet files. + * @return A list of attributes that make up the schema. + */ + def readSchemaFromFile(origPath: Path): Seq[Attribute] = { + val keyValueMetadata: java.util.Map[String, String] = + readMetaData(origPath) + .getFileMetaData + .getKeyValueMetaData + if (keyValueMetadata.get(RowReadSupport.SPARK_METADATA_KEY) != null) { + convertFromString(keyValueMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) + } else { + val attributes = convertToAttributes( + readMetaData(origPath).getFileMetaData.getSchema) + log.warn(s"Falling back to schema conversion from Parquet types; result: $attributes") + attributes + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 950f729dba0ae..72aa07d79928d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -111,10 +111,23 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { } test("Read/Write All Types") { - val data = AllDataTypes("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true) val tempDir = getTempFilePath("parquetTest").getCanonicalPath - sparkContext.parallelize(data :: Nil).saveAsParquetFile(tempDir) - assert(parquetFile(tempDir).collect().head === data) + val range = (0 to 255) + TestSQLContext.sparkContext.parallelize(range) + .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + .saveAsParquetFile(tempDir) + val result = parquetFile(tempDir).collect() + range.foreach { + i => + assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") + assert(result(i).getInt(1) === i) + assert(result(i).getLong(2) === i.toLong) + assert(result(i).getFloat(3) === i.toFloat) + assert(result(i).getDouble(4) === i.toDouble) + assert(result(i).getShort(5) === i.toShort) + assert(result(i).getByte(6) === i.toByte) + assert(result(i).getBoolean(7) === (i % 2 == 0)) + } } test("self-join parquet files") { From 403061fc73400b0bd6fc8d5a7811e21130f245c8 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 8 Jun 2014 21:32:03 +0300 Subject: [PATCH 44/48] Fixing some issues with tests and schema metadata --- .../sql/parquet/ParquetTableSupport.scala | 34 +++++++++++++------ .../spark/sql/parquet/ParquetQuerySuite.scala | 16 +++++++-- 2 files changed, 36 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index f4ce210266fcd..ffc1949b771de 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 @@ -58,18 +58,31 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { stringMap: java.util.Map[String, String], fileSchema: MessageType, readContext: ReadContext): RecordMaterializer[Row] = { - log.debug(s"preparing for read with file schema $fileSchema") + 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] = - if (readContext.getReadSupportMetadata != null && - readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_METADATA_KEY) != null) { - ParquetTypesConverter.convertFromString( - readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) + 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 { - // fall back to converting from Parquet schema - ParquetTypesConverter.convertToAttributes(parquetSchema) + // 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)) + } } + } + // if both unavailable, fall back to deducing the schema from the given Parquet schema + if (schema == null) { + log.debug("falling back to Parquet read schema") + schema = ParquetTypesConverter.convertToAttributes(parquetSchema) + } + log.debug(s"list of attributes that will be read: $schema") new RowRecordMaterializer(parquetSchema, schema) } @@ -78,17 +91,16 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { keyValueMetaData: java.util.Map[String, String], fileSchema: MessageType): ReadContext = { var parquetSchema: MessageType = fileSchema - var metadata: java.util.Map[String, String] = null + var metadata: java.util.Map[String, String] = new java.util.HashMap[String, String]() val requestedAttributes = RowReadSupport.getRequestedSchema(configuration) if (requestedAttributes != null) { parquetSchema = ParquetTypesConverter.convertFromAttributes(requestedAttributes) + metadata.put(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertToString(requestedAttributes)) } val origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) - if (origAttributesStr != null) { - metadata = new java.util.HashMap[String, String]() metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 72aa07d79928d..6e7432c2557ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -77,7 +77,7 @@ case class AllDataTypes( byteField: Byte, booleanField: Boolean) -class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { +class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { import TestData._ TestData // Load test data tables. @@ -264,14 +264,24 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { assert(rdd_copy1.size === 100) assert(rdd_copy1(0).apply(0) === 1) assert(rdd_copy1(0).apply(1) === "val_1") - sql("INSERT INTO dest SELECT * FROM source").collect() + // TODO: why does collecting break things? It seems InsertIntoParquet::execute() is + // executed twice otherwise?! + sql("INSERT INTO dest SELECT * FROM source") val rdd_copy2 = sql("SELECT * FROM dest").collect() assert(rdd_copy2.size === 200) + assert(rdd_copy2(0).apply(0) === 1) + assert(rdd_copy2(0).apply(1) === "val_1") + assert(rdd_copy2(99).apply(0) === 100) + assert(rdd_copy2(99).apply(1) === "val_100") + assert(rdd_copy2(100).apply(0) === 1) + assert(rdd_copy2(100).apply(1) === "val_1") Utils.deleteRecursively(dirname) } test("Insert (appending) to same table via Scala API") { - sql("INSERT INTO testsource SELECT * FROM testsource").collect() + // TODO: why does collecting break things? It seems InsertIntoParquet::execute() is + // executed twice otherwise?! + sql("INSERT INTO testsource SELECT * FROM testsource") val double_rdd = sql("SELECT * FROM testsource").collect() assert(double_rdd != null) assert(double_rdd.size === 30) From 94eea3a640892b8f96a4b297f013276010f4de65 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Sun, 8 Jun 2014 21:34:59 +0300 Subject: [PATCH 45/48] Scalastyle --- .../spark/sql/parquet/ParquetTableSupport.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index ffc1949b771de..bfcbdeb34a92f 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 @@ -65,12 +65,15 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { 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) { + 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 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)) @@ -96,7 +99,9 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { if (requestedAttributes != null) { parquetSchema = ParquetTypesConverter.convertFromAttributes(requestedAttributes) - metadata.put(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertToString(requestedAttributes)) + metadata.put( + RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, + ParquetTypesConverter.convertToString(requestedAttributes)) } val origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) From 7eceb679b7cffc0721d13a590853ba1310c3c911 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 19 Jun 2014 18:43:51 +0300 Subject: [PATCH 46/48] Review feedback --- project/SparkBuild.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 44 +++++++++---------- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 605fd4839fa7d..382d1843b24e7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -505,7 +505,7 @@ object SparkBuild extends Build { "com.twitter" % "parquet-hadoop" % parquetVersion, "com.twitter" % "parquet-avro" % parquetVersion % "test", // here we need version >= 1.7.5 because of AVRO-1274 - "org.apache.avro" % "avro" % "1.7.6" % "test" + "org.apache.avro" % "avro" % "1.7.6" % "test", "com.fasterxml.jackson.core" % "jackson-databind" % "2.3.0" // json4s-jackson 3.2.6 requires jackson-databind 2.3.0. ), initialCommands in console := 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 9f72d72cd7437..889a408e3c393 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 @@ -83,15 +83,15 @@ private[sql] object CatalystConverter { new CatalystArrayConverter(elementType, fieldIndex, parent) } case StructType(fields: Seq[StructField]) => { - new CatalystStructConverter(fields, fieldIndex, parent) + new CatalystStructConverter(fields.toArray, fieldIndex, parent) } case MapType(keyType: DataType, valueType: DataType) => { new CatalystMapConverter( - Seq( + Array( new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false), new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)), - fieldIndex, - parent) + fieldIndex, + parent) } // Strings, Shorts and Bytes do not have a corresponding type in Parquet // so we need to treat them separately @@ -127,14 +127,14 @@ private[sql] object CatalystConverter { 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) + new CatalystPrimitiveRowConverter(attributes.toArray) } else { - new CatalystGroupConverter(attributes) + new CatalystGroupConverter(attributes.toArray) } } } -private[parquet] trait CatalystConverter { +private[parquet] abstract class CatalystConverter extends GroupConverter { /** * The number of fields this group has */ @@ -206,14 +206,14 @@ private[parquet] trait CatalystConverter { * @param schema The corresponding Catalyst schema in the form of a list of attributes. */ private[parquet] class CatalystGroupConverter( - protected[parquet] val schema: Seq[FieldType], + 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[Row]) - extends GroupConverter with CatalystConverter { + extends CatalystConverter { - def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) = + def this(schema: Array[FieldType], index: Int, parent: CatalystConverter) = this( schema, index, @@ -225,7 +225,7 @@ private[parquet] class CatalystGroupConverter( /** * This constructor is used for the root converter only! */ - def this(attributes: Seq[Attribute]) = + 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] = @@ -254,7 +254,7 @@ private[parquet] class CatalystGroupConverter( override protected[parquet] def clearBuffer(): Unit = buffer.clear() override def start(): Unit = { - current = ArrayBuffer.fill(schema.length)(null) + current = ArrayBuffer.fill(size)(null) converters.foreach { converter => if (!converter.isPrimitive) { converter.asInstanceOf[CatalystConverter].clearBuffer @@ -277,12 +277,12 @@ private[parquet] class CatalystGroupConverter( * converter is optimized for rows of primitive types (non-nested records). */ private[parquet] class CatalystPrimitiveRowConverter( - protected[parquet] val schema: Seq[FieldType], + protected[parquet] val schema: Array[FieldType], protected[parquet] var current: ParquetRelation.RowType) - extends GroupConverter with CatalystConverter { + extends CatalystConverter { // This constructor is used for the root converter only - def this(attributes: Seq[Attribute]) = + def this(attributes: Array[Attribute]) = this( attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), new ParquetRelation.RowType(attributes.length)) @@ -313,7 +313,7 @@ private[parquet] class CatalystPrimitiveRowConverter( override def start(): Unit = { var i = 0 - while (i < schema.length) { + while (i < size) { current.setNullAt(i) i = i + 1 } @@ -398,7 +398,7 @@ private[parquet] class CatalystArrayConverter( val index: Int, protected[parquet] val parent: CatalystConverter, protected[parquet] var buffer: Buffer[Any]) - extends GroupConverter with CatalystConverter { + extends CatalystConverter { def this(elementType: DataType, index: Int, parent: CatalystConverter) = this( @@ -462,7 +462,7 @@ private[parquet] class CatalystNativeArrayConverter( val index: Int, protected[parquet] val parent: CatalystConverter, protected[parquet] var capacity: Int = CatalystArrayConverter.INITIAL_ARRAY_SIZE) - extends GroupConverter with CatalystConverter { + extends CatalystConverter { type NativeType = elementType.JvmType @@ -577,7 +577,7 @@ private[parquet] class CatalystNativeArrayConverter( * @param parent */ private[parquet] class CatalystStructConverter( - override protected[parquet] val schema: Seq[FieldType], + override protected[parquet] val schema: Array[FieldType], override protected[parquet] val index: Int, override protected[parquet] val parent: CatalystConverter) extends CatalystGroupConverter(schema, index, parent) { @@ -605,14 +605,14 @@ private[parquet] class CatalystStructConverter( * @param parent */ private[parquet] class CatalystMapConverter( - protected[parquet] val schema: Seq[FieldType], + protected[parquet] val schema: Array[FieldType], override protected[parquet] val index: Int, override protected[parquet] val parent: CatalystConverter) - extends GroupConverter with CatalystConverter { + extends CatalystConverter { private val map = new HashMap[Any, Any]() - private val keyValueConverter = new GroupConverter with CatalystConverter { + private val keyValueConverter = new CatalystConverter { private var currentKey: Any = null private var currentValue: Any = null val keyConverter = CatalystConverter.createConverter(schema(0), 0, this) From 95c1367e818c7a5958959751c9df462968ab0ebd Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 19 Jun 2014 19:27:02 +0300 Subject: [PATCH 47/48] Changes to ParquetRelation and its metadata The Hadoop configuration is now passed to ParquetRelation (fixes SPARK-2112) and the path is no longer stored in metadata (fixes SPARK-2195) --- .../scala/org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/api/java/JavaSQLContext.scala | 4 +++- .../spark/sql/parquet/ParquetRelation.scala | 11 ++++++----- .../apache/spark/sql/parquet/ParquetTypes.scala | 15 +++++++-------- .../spark/sql/parquet/ParquetQuerySuite.scala | 2 +- 5 files changed, 18 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 1617ec717b2e0..ab376e5504d35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -94,7 +94,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def parquetFile(path: String): SchemaRDD = - new SchemaRDD(this, parquet.ParquetRelation(path)) + new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration))) /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index ff9842267ffe0..ff6deeda2394d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -99,7 +99,9 @@ class JavaSQLContext(val sqlContext: SQLContext) { * Loads a parquet file, returning the result as a [[JavaSchemaRDD]]. */ def parquetFile(path: String): JavaSchemaRDD = - new JavaSchemaRDD(sqlContext, ParquetRelation(path)) + new JavaSchemaRDD( + sqlContext, + ParquetRelation(path, Some(sqlContext.sparkContext.hadoopConfiguration))) /** * Loads a JSON file (one object per line), returning the result as a [[JavaSchemaRDD]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 67edfd8a19025..96c131a7f8af1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -43,19 +43,20 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} * * @param path The path to the Parquet file. */ -private[sql] case class ParquetRelation(val path: String) - extends LeafNode with MultiInstanceRelation { +private[sql] case class ParquetRelation( + val path: String, + @transient val conf: Option[Configuration] = None) extends LeafNode with MultiInstanceRelation { self: Product => /** Schema derived from ParquetFile */ def parquetSchema: MessageType = ParquetTypesConverter - .readMetaData(new Path(path)) + .readMetaData(new Path(path), conf) .getFileMetaData .getSchema /** Attributes */ - override val output = ParquetTypesConverter.readSchemaFromFile(new Path(path)) + override val output = ParquetTypesConverter.readSchemaFromFile(new Path(path), conf) override def newInstance = ParquetRelation(path).asInstanceOf[this.type] @@ -130,7 +131,7 @@ private[sql] object ParquetRelation { } ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) - new ParquetRelation(path.toString) { + new ParquetRelation(path.toString, Some(conf)) { override val output = attributes } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 925da184f3d3b..f9046368e7ced 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -322,7 +322,6 @@ private[parquet] object ParquetTypesConverter extends Logging { } } val extraMetadata = new java.util.HashMap[String, String]() - extraMetadata.put("path", path.toString) extraMetadata.put( RowReadSupport.SPARK_METADATA_KEY, ParquetTypesConverter.convertToString(attributes)) @@ -347,16 +346,15 @@ private[parquet] object ParquetTypesConverter extends Logging { * in the parent directory. If so, this is used. Else we read the actual footer at the given * location. * @param origPath The path at which we expect one (or more) Parquet files. + * @param configuration The Hadoop configuration to use. * @return The `ParquetMetadata` containing among other things the schema. */ - def readMetaData(origPath: Path): ParquetMetadata = { + def readMetaData(origPath: Path, configuration: Option[Configuration]): ParquetMetadata = { if (origPath == null) { throw new IllegalArgumentException("Unable to read Parquet metadata: path is null") } val job = new Job() - // TODO: since this is called from ParquetRelation (LogicalPlan) we don't have access - // to SparkContext's hadoopConfig; in principle the default FileSystem may be different(?!) - val conf = ContextUtil.getConfiguration(job) + val conf = configuration.getOrElse(ContextUtil.getConfiguration(job)) val fs: FileSystem = origPath.getFileSystem(conf) if (fs == null) { throw new IllegalArgumentException(s"Incorrectly formatted Parquet metadata path $origPath") @@ -390,18 +388,19 @@ private[parquet] object ParquetTypesConverter extends Logging { * may lead to an upcast of types (e.g., {byte, short} to int). * * @param origPath The path at which we expect one (or more) Parquet files. + * @param conf The Hadoop configuration to use. * @return A list of attributes that make up the schema. */ - def readSchemaFromFile(origPath: Path): Seq[Attribute] = { + def readSchemaFromFile(origPath: Path, conf: Option[Configuration]): Seq[Attribute] = { val keyValueMetadata: java.util.Map[String, String] = - readMetaData(origPath) + readMetaData(origPath, conf) .getFileMetaData .getKeyValueMetaData if (keyValueMetadata.get(RowReadSupport.SPARK_METADATA_KEY) != null) { convertFromString(keyValueMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) } else { val attributes = convertToAttributes( - readMetaData(origPath).getFileMetaData.getSchema) + readMetaData(origPath, conf).getFileMetaData.getSchema) log.warn(s"Falling back to schema conversion from Parquet types; result: $attributes") attributes } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 6e7432c2557ab..d64d38f36ba24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -207,7 +207,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA path, TestSQLContext.sparkContext.hadoopConfiguration) assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) - val metaData = ParquetTypesConverter.readMetaData(path) + val metaData = ParquetTypesConverter.readMetaData(path, Some(ContextUtil.getConfiguration(job))) assert(metaData != null) ParquetTestData .testData From 30708c864b656f04d046a1d3be4e9751960a1cb7 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 20 Jun 2014 08:21:13 +0300 Subject: [PATCH 48/48] Taking out AvroParquet test for now to remove Avro dependency --- project/SparkBuild.scala | 3 - .../spark/sql/parquet/ParquetQuerySuite.scala | 147 ------------------ 2 files changed, 150 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 382d1843b24e7..7bb39dc77120b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -503,9 +503,6 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.twitter" % "parquet-column" % parquetVersion, "com.twitter" % "parquet-hadoop" % parquetVersion, - "com.twitter" % "parquet-avro" % parquetVersion % "test", - // here we need version >= 1.7.5 because of AVRO-1274 - "org.apache.avro" % "avro" % "1.7.6" % "test", "com.fasterxml.jackson.core" % "jackson-databind" % "2.3.0" // json4s-jackson 3.2.6 requires jackson-databind 2.3.0. ), initialCommands in console := diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index d64d38f36ba24..0c239d00b199b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -17,17 +17,11 @@ package org.apache.spark.sql.parquet -import java.io.File - import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} -import org.apache.avro.{SchemaBuilder, Schema} -import org.apache.avro.generic.{GenericData, GenericRecord} - import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.mapreduce.Job -import parquet.avro.AvroParquetWriter import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil import parquet.schema.MessageTypeParser @@ -679,147 +673,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } - - test("Importing data generated with Avro") { - val tmpdir = Utils.createTempDir() - val file: File = new File(tmpdir, "test.avro") - - val primitiveArrayType: Schema = SchemaBuilder.array.items.intType - val complexArrayType: Schema = SchemaBuilder.array.items.map.values.stringType - val primitiveMapType: Schema = SchemaBuilder.map.values.booleanType - val complexMapType: Schema = SchemaBuilder.map.values.array.items.floatType - val schema: Schema = SchemaBuilder - .record("TestRecord") - .namespace("") - .fields - .name("testInt") - .`type`. - intType - .noDefault - .name("testDouble") - .`type` - .doubleType - .noDefault - .name("testString") - .`type` - .nullable - .stringType - .stringDefault("") - .name("testPrimitiveArray") - .`type`(primitiveArrayType) - .noDefault - .name("testComplexArray") - .`type`(complexArrayType) - .noDefault - .name("testPrimitiveMap") - .`type`(primitiveMapType) - .noDefault - .name("testComplexMap") - .`type`(complexMapType) - .noDefault - .endRecord - - val record1: GenericRecord = new GenericData.Record(schema) - - // primitive fields - record1.put("testInt", 256) - record1.put("testDouble", 0.5) - record1.put("testString", "foo") - - val primitiveArrayData = new GenericData.Array[Integer](10, primitiveArrayType) - val complexArrayData: GenericData.Array[java.util.Map[String, String]] = - new GenericData.Array[java.util.Map[String, String]](10, SchemaBuilder.array.items.map.values.stringType) - - // two arrays: one primitive (array of ints), one complex (array of string->string maps) - primitiveArrayData.add(1) - primitiveArrayData.add(2) - primitiveArrayData.add(3) - val map1 = new java.util.HashMap[String, String] - map1.put("key11", "data11") - map1.put("key12", "data12") - val map2 = new java.util.HashMap[String, String] - map2.put("key21", "data21") - map2.put("key22", "data22") - complexArrayData.add(0, map1) - complexArrayData.add(1, map2) - - record1.put("testPrimitiveArray", primitiveArrayData) - record1.put("testComplexArray", complexArrayData) - - // two maps: one primitive (string->boolean), one complex (string->array of floats) - val primitiveMap = new java.util.HashMap[String, Boolean](10) - primitiveMap.put("key1", true) - primitiveMap.put("key2", false) - val complexMap = new java.util.HashMap[String, GenericData.Array[Float]](10) - val value1: GenericData.Array[Float] = new GenericData.Array[Float](10, SchemaBuilder.array.items.floatType) - value1.add(0.1f) - value1.add(0.2f) - value1.add(0.3f) - complexMap.put("compKey1", value1) - val value2: GenericData.Array[Float] = new GenericData.Array[Float](10, SchemaBuilder.array.items.floatType) - value2.add(1.1f) - value2.add(1.2f) - value2.add(1.3f) - complexMap.put("compKey2", value2) - - record1.put("testPrimitiveMap", primitiveMap) - record1.put("testComplexMap", complexMap) - - // TODO: test array or map with value type Avro record - - val writer = new AvroParquetWriter[GenericRecord](new Path(file.toString), schema) - writer.write(record1) - writer.close() - - val data = TestSQLContext - .parquetFile(tmpdir.toString) - .toSchemaRDD - data.registerAsTable("avroTable") - val resultPrimitives = sql("SELECT testInt, testDouble, testString FROM avroTable").collect() - assert(resultPrimitives(0)(0) === 256) - assert(resultPrimitives(0)(1) === 0.5) - assert(resultPrimitives(0)(2) === "foo") - val resultPrimitiveArray = sql("SELECT testPrimitiveArray FROM avroTable").collect() - assert(resultPrimitiveArray(0)(0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 1) - assert(resultPrimitiveArray(0)(0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](1) === 2) - assert(resultPrimitiveArray(0)(0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](2) === 3) - val resultComplexArray = sql("SELECT testComplexArray FROM avroTable").collect() - assert(resultComplexArray(0)(0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]].size === 2) - assert( - resultComplexArray(0)(0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) - .asInstanceOf[CatalystConverter.MapScalaType[String, String]] - .get("key11").get.equals("data11")) - assert( - resultComplexArray(0)(0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](1) - .asInstanceOf[CatalystConverter.MapScalaType[String, String]] - .get("key22").get.equals("data22")) - val resultPrimitiveMap = sql("SELECT testPrimitiveMap FROM avroTable").collect() - assert( - resultPrimitiveMap(0)(0) - .asInstanceOf[CatalystConverter.MapScalaType[String, Boolean]].get("key1").get === true) - assert( - resultPrimitiveMap(0)(0) - .asInstanceOf[CatalystConverter.MapScalaType[String, Boolean]].get("key2").get === false) - val resultComplexMap = sql("SELECT testComplexMap FROM avroTable").collect() - val mapResult1 = - resultComplexMap(0)(0) - .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.ArrayScalaType[_]]] - .get("compKey1").get - val mapResult2 = - resultComplexMap(0)(0) - .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.ArrayScalaType[_]]] - .get("compKey2").get - assert(mapResult1(0) === 0.1f) - assert(mapResult1(2) === 0.3f) - assert(mapResult2(0) === 1.1f) - assert(mapResult2(2) === 1.3f) - } } // TODO: the code below is needed temporarily until the standard parser is able to parse