You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by AndreSchumacher <gi...@git.apache.org> on 2014/04/08 17:20:26 UTC

[GitHub] spark pull request: SPARK-1293 [SQL] WIP Parquet support for neste...

GitHub user AndreSchumacher opened a pull request:

    https://github.com/apache/spark/pull/360

    SPARK-1293 [SQL] WIP Parquet support for nested types

    It should be possible to import and export data stored in Parquet's columnar format that contains nested types. For example:
    ```java
    message AddressBook {
       required string owner;
       optional group ownerPhoneNumbers {
          repeated string values;
       }
       repeated group contacts {
          required string name;
          optional string phoneNumber;
       }
    }
    ```
    The example could model a type (AddressBook) that contains records made of strings (owner), lists (ownerPhoneNumbers) and a table of contacts (e.g., a list of pairs of a map). The list of tasks are as follows:
    
    <h6>Implement support for converting nested Parquet types to Spark/Catalyst types:</h6>
    - [x] Structs
    - [x] Lists
    - [ ] Maps
    
    <h6>Implement import (via ``parquetFile``) of nested Parquet types (first version in this PR)</h6>
    - [x] Initial version (without maps)
     
    <h6>Implement export (via ``saveAsParquetFile``)</h6>
    - [ ] Initial version (missing)
    
    <h6>Test support for AvroParquet, etc.</h6>
    
    Example:
    ```scala
    val data = TestSQLContext
      .parquetFile("input.dir")
      .toSchemaRDD
    data.registerAsTable("data")
    sql("SELECT owner, contacts[1].name FROM data").collect()
    ```

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/AndreSchumacher/spark nested_parquet

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/360.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #360
    
----
commit 7318fe19eac8caf471feec8e4830a538aa647770
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-03-26T07:46:10Z

    Adding conversion of nested Parquet schemas

commit 0649f3b407632041df63d3306773b657255dbcb3
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-03-27T16:24:13Z

    First commit nested Parquet read converters

commit 341d7e55e7a66e04f0ce45b5b1fa9f8cb7debbeb
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-03-27T17:48:16Z

    First working nested Parquet record input

commit 832d263c1056efe04fe353b9718ce3f0ad307c28
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-01T13:17:02Z

    Completing testcase for nested data (Addressbook(

commit 7f5bd07876aa2a228db67b3bd7d2baa938d1c79c
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-01T14:15:23Z

    Extending tests for nested Parquet data

commit e9da236fdad31071fddd668dde3b7c303cd08d79
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-02T12:42:19Z

    Fixing one problem with nested arrays

commit e4375db6d50baf4f629dd71e82b92881841c3b04
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-02T14:00:46Z

    fixing one problem with nested structs and breaking up files

commit 7c4e79aa61fcbeba0f06c8e40e23a2f486e0cce8
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-02T14:45:22Z

    added struct converter

commit 04e97d1c355e054c9db51766e2582700f299751e
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-03T15:11:40Z

    fixing one problem with arrayconverter

commit 0cc0edb93f5b89197697286ec9cc705cd8fd5edf
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-04T16:56:56Z

    Documenting conversions, bugfix, wrappers of Rows

commit 0fae86af7a463bb2ad04db571c970b85bc6de333
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-06T14:19:23Z

    Fixing some problems intruduced during rebase

commit 2dc7adc23deb1ba05bef123db08b939a0d386082
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-06T16:04:44Z

    For primitive rows fall back to more efficient converter, code reorg

commit 8df7d0c1c710bb44fa904165f6b0352732c83468
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-08T07:27:26Z

    Adding resolution of complex ArrayTypes

commit 79b6a7a1c126e54bbd31614b202e39fc1d882e93
Author: Andre Schumacher <an...@iki.fi>
Date:   2014-04-08T14:55:46Z

    Scalastyle

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932798
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala ---
    @@ -153,9 +153,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")
    --- End diff --
    
    Probably should not be info.  Also, why do all the data types have to be primitive for us to use mutable rows?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932653
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,582 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +private[parquet] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            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")
    +    }
    +  }
    +
    +  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 PrimitiveRowGroupConverter(attributes)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  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
    +
    +  // Should be only called in root group converter!
    +  def getCurrentRecord: Row
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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!")
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class PrimitiveRowGroupConverter(
    +    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.
    + */
    +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
    +}
    +
    +/**
    + * 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 (complex or primitive)
    + * @param index The position of this (array) field inside its parent converter
    + * @param parent The parent converter
    + * @param buffer A data buffer
    + */
    +private[parquet] class CatalystArrayConverter(
    +    val elementType: DataType,
    +    val index: Int,
    +    protected[parquet] val parent: CatalystConverter,
    +    protected[parquet] var buffer: Buffer[Any])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(elementType: DataType, index: Int, parent: CatalystConverter) =
    +    this(
    +      elementType,
    +      index,
    +      parent,
    +      new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE))
    +
    +  protected[parquet] val converter: Converter = CatalystConverter.createConverter(
    +    new CatalystConverter.FieldType(
    +      CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +      elementType,
    +      false),
    +    fieldIndex=0,
    +    parent=this)
    +
    +  override def getConverter(fieldIndex: Int): Converter = converter
    +
    +  // arrays have only one (repeated) field, which is its elements
    +  override val size = 1
    +
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    // fieldIndex is ignored (assumed to be zero but not checked)
    +    buffer += value
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = {
    +    buffer.clear()
    +  }
    +
    +  override def start(): Unit = {
    +    if (!converter.isPrimitive) {
    +      converter.asInstanceOf[CatalystConverter].clearBuffer
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    assert(parent != null)
    +    parent.updateField(index, new GenericRow(buffer.toArray))
    +    clearBuffer()
    +  }
    +
    +  // Should be only called in root group converter!
    +  override def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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 {
    +
    +  type nativeType = elementType.JvmType
    +
    +  private var buffer: Array[nativeType] = elementType.classTag.newArray(capacity)
    +
    +  private var elements: Int = 0
    +
    +  protected[parquet] val converter: Converter = CatalystConverter.createConverter(
    +    new CatalystConverter.FieldType(
    +      CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +      elementType,
    +      false),
    +    fieldIndex=0,
    +    parent=this)
    +
    +  override def getConverter(fieldIndex: Int): Converter = converter
    +
    +  // arrays have only one (repeated) field, which is its elements
    +  override val size = 1
    +
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit =
    +    throw new UnsupportedOperationException
    +
    +  // 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]
    --- End diff --
    
    Can you have nulls in parquet arrays?  If so I'm afraid we are loosing them here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40242736
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14056/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13876519
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,667 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +/**
    + * Collection of converters of Parquet types (group and primitive types) that
    + * model arrays and maps. The conversions are partly based on the AvroParquet
    + * converters that are part of Parquet in order to be able to process these
    + * types.
    + *
    + * There are several types of converters:
    + * <ul>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive
    + *   (numeric, boolean and String) types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays
    + *   of native JVM element types; note: currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of
    + *   arbitrary element types (including nested element types); note: currently
    + *   null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note:
    + *   currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows
    + *   of only primitive element types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested
    + *   records, including the top-level row record</li>
    + * </ul>
    + */
    +
    +private[sql] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  // 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]
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            fieldIndex,
    +            parent)
    +      }
    +      // 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")
    +    }
    +  }
    +
    +  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)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  /**
    +   * Called by child converters to update their value in its parent (this).
    +   * Note that if possible the more specific update methods below should be used
    +   * to avoid auto-boxing of native JVM types.
    +   *
    +   * @param fieldIndex
    +   * @param value
    +   */
    +  protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit
    +
    +  protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.getBytes)
    +
    +  protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.toStringUsingUTF8)
    +
    +  protected[parquet] def isRootConverter: Boolean = parent == null
    +
    +  protected[parquet] def clearBuffer(): Unit
    +
    +  /**
    +   * Should only be called in the root (group) converter!
    +   *
    +   * @return
    +   */
    +  def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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
    +
    +  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
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class CatalystPrimitiveRowConverter(
    +    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!
    +  override 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) {
    --- End diff --
    
    schema.length slows things down substantially in our profiling. should just put it in a variable


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13892570
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,667 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +/**
    + * Collection of converters of Parquet types (group and primitive types) that
    + * model arrays and maps. The conversions are partly based on the AvroParquet
    + * converters that are part of Parquet in order to be able to process these
    + * types.
    + *
    + * There are several types of converters:
    + * <ul>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive
    + *   (numeric, boolean and String) types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays
    + *   of native JVM element types; note: currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of
    + *   arbitrary element types (including nested element types); note: currently
    + *   null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note:
    + *   currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows
    + *   of only primitive element types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested
    + *   records, including the top-level row record</li>
    + * </ul>
    + */
    +
    +private[sql] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  // 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]
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            fieldIndex,
    +            parent)
    +      }
    +      // 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")
    +    }
    +  }
    +
    +  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)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  /**
    +   * Called by child converters to update their value in its parent (this).
    +   * Note that if possible the more specific update methods below should be used
    +   * to avoid auto-boxing of native JVM types.
    +   *
    +   * @param fieldIndex
    +   * @param value
    +   */
    +  protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit
    +
    +  protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.getBytes)
    +
    +  protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.toStringUsingUTF8)
    +
    +  protected[parquet] def isRootConverter: Boolean = parent == null
    +
    +  protected[parquet] def clearBuffer(): Unit
    +
    +  /**
    +   * Should only be called in the root (group) converter!
    +   *
    +   * @return
    +   */
    +  def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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
    +
    +  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
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class CatalystPrimitiveRowConverter(
    +    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!
    +  override 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) {
    --- End diff --
    
    But yes in general it would be better to have the inner loop code operating on arrays rather than any Scala data structures.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46645804
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40242735
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11986096
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala ---
    @@ -52,6 +52,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression {
           }
         } else {
           val baseValue = child.eval(input).asInstanceOf[Map[Any, _]]
    +      // TODO: recover key type!!
    --- End diff --
    
    `MapType` records the type of the key. I was wondering whether one should use that instead if possible and not just Any. The comment relates to the other comments inside the resolver.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] WIP Parquet support for neste...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-39866399
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-41234534
  
    Wow, this is a pretty intense PR! :)
    
    Still trying to wrap my head around it, but overall I think the approach seems reasonable.  Will take another look after a few questions are answered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932422
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.parquet
    +
    +import java.io.IOException
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.mapreduce.Job
    +
    +import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter}
    +import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData}
    +import parquet.hadoop.util.ContextUtil
    +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser}
    +import parquet.schema.{GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns}
    +import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName}
    +import parquet.schema.Type.Repetition
    +
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute}
    +import org.apache.spark.sql.catalyst.types._
    +
    +// Implicits
    +import scala.collection.JavaConversions._
    +
    +private[parquet] object ParquetTypesConverter {
    +  def isPrimitiveType(ctype: DataType): Boolean =
    +    classOf[PrimitiveType] isAssignableFrom ctype.getClass
    +
    +  def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match {
    +    case ParquetPrimitiveTypeName.BINARY => StringType
    +    case ParquetPrimitiveTypeName.BOOLEAN => BooleanType
    +    case ParquetPrimitiveTypeName.DOUBLE => DoubleType
    +    case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType)
    +    case ParquetPrimitiveTypeName.FLOAT => FloatType
    +    case ParquetPrimitiveTypeName.INT32 => IntegerType
    +    case ParquetPrimitiveTypeName.INT64 => LongType
    +    case ParquetPrimitiveTypeName.INT96 =>
    +      // TODO: add BigInteger type? TODO(andre) use DecimalType instead????
    +      sys.error("Potential loss of precision: cannot convert INT96")
    +    case _ => sys.error(
    +      s"Unsupported parquet datatype $parquetType")
    +  }
    +
    +  /**
    +   * Converts a given Parquet `Type` into the corresponding
    +   * [[org.apache.spark.sql.catalyst.types.DataType]].
    +   *
    +   * Note that we apply the following conversion rules:
    +   * <ul>
    +   *   <li> Primitive types are converter to the corresponding primitive type.</li>
    +   *   <li> Group types that have a single field that is itself a group, which has repetition
    +   *        level `REPEATED`, are treated as follows:<ul>
    +   *          <li> If the nested group has name `values`, the surrounding group is converted
    +   *               into an [[ArrayType]] with the corresponding field type (primitive or
    +   *               complex) as element type.</li>
    +   *          <li> If the nested group has name `map` and two fields (named `key` and `value`),
    +   *               the surrounding group is converted into a [[MapType]]
    +   *               with the corresponding key and value (value possibly complex) types.
    +   *               Note that we currently assume map values are not nullable.</li>
    +   *   <li> Other group types are converted into a [[StructType]] with the corresponding
    +   *        field types.</li></ul></li>
    +   * </ul>
    +   * Note that fields are determined to be `nullable` if and only if their Parquet repetition
    +   * level is not `REQUIRED`.
    +   *
    +   * @param parquetType The type to convert.
    +   * @return The corresponding Catalyst type.
    +   */
    +  def toDataType(parquetType: ParquetType): DataType = {
    +    def correspondsToMap(groupType: ParquetGroupType): Boolean = {
    +      if (groupType.getFieldCount != 1 || groupType.getFields.apply(0).isPrimitive) {
    +        false
    +      } else {
    +        // This mostly follows the convention in ``parquet.schema.ConversionPatterns``
    +        val keyValueGroup = groupType.getFields.apply(0).asGroupType()
    +        keyValueGroup.getRepetition == Repetition.REPEATED &&
    +          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.getFieldName(0) == CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME &&
    +        groupType.getFields.apply(0).getRepetition == Repetition.REPEATED
    +    }
    +
    +    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!
    +          assert(groupType.getFieldCount == 1)
    +          val field = groupType.getFields.apply(0)
    +          new ArrayType(toDataType(field))
    +        }
    +        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))
    +          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 _ => {
    +          // 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))
    +            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))
    +            new ArrayType(elementType)
    +          } else { // everything else: StructType
    +            val fields = groupType
    +              .getFields
    +              .map(ptype => new StructField(
    +              ptype.getName,
    +              toDataType(ptype),
    +              ptype.getRepetition != Repetition.REQUIRED))
    +            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`.
    --- End diff --
    
    "corresponding"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13892425
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,667 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +/**
    + * Collection of converters of Parquet types (group and primitive types) that
    + * model arrays and maps. The conversions are partly based on the AvroParquet
    + * converters that are part of Parquet in order to be able to process these
    + * types.
    + *
    + * There are several types of converters:
    + * <ul>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive
    + *   (numeric, boolean and String) types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays
    + *   of native JVM element types; note: currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of
    + *   arbitrary element types (including nested element types); note: currently
    + *   null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note:
    + *   currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows
    + *   of only primitive element types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested
    + *   records, including the top-level row record</li>
    + * </ul>
    + */
    +
    +private[sql] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  // 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]
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            fieldIndex,
    +            parent)
    +      }
    +      // 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")
    +    }
    +  }
    +
    +  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)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] trait CatalystConverter {
    --- End diff --
    
    The abstraction overhead of a trait is nontrivially high, and also seemingly unnecessary here. Please make this an abstract class that extends GroupConverter. I achieved around an 8-10% performance improvement in a benchmark that reads several billion Longs from a particular column with this change.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45140163
  
     Build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932307
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala ---
    @@ -29,25 +31,36 @@ abstract class DataType {
         case e: Expression if e.dataType == this => true
         case _ => false
       }
    +
    +  def isPrimitive(): Boolean = false
     }
     
     case object NullType extends DataType
     
    +trait PrimitiveType extends DataType {
    --- End diff --
    
    What are the semantics of `PrimitiveType`? Specifically, I'm surprised that `StringType` and `DecimalType` are considered `PrimitiveTypes`.  Also I wonder if we can unify this with `NativeType` somehow.  I'm not really sure, but I'd like to avoid too much explosion here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40316177
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14099/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13892309
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,667 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +/**
    + * Collection of converters of Parquet types (group and primitive types) that
    + * model arrays and maps. The conversions are partly based on the AvroParquet
    + * converters that are part of Parquet in order to be able to process these
    + * types.
    + *
    + * There are several types of converters:
    + * <ul>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive
    + *   (numeric, boolean and String) types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays
    + *   of native JVM element types; note: currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of
    + *   arbitrary element types (including nested element types); note: currently
    + *   null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note:
    + *   currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows
    + *   of only primitive element types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested
    + *   records, including the top-level row record</li>
    + * </ul>
    + */
    +
    +private[sql] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  // 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]
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            fieldIndex,
    +            parent)
    +      }
    +      // 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")
    +    }
    +  }
    +
    +  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)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  /**
    +   * Called by child converters to update their value in its parent (this).
    +   * Note that if possible the more specific update methods below should be used
    +   * to avoid auto-boxing of native JVM types.
    +   *
    +   * @param fieldIndex
    +   * @param value
    +   */
    +  protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit
    +
    +  protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.getBytes)
    +
    +  protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.toStringUsingUTF8)
    +
    +  protected[parquet] def isRootConverter: Boolean = parent == null
    +
    +  protected[parquet] def clearBuffer(): Unit
    +
    +  /**
    +   * Should only be called in the root (group) converter!
    +   *
    +   * @return
    +   */
    +  def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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
    +
    +  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
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class CatalystPrimitiveRowConverter(
    +    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!
    +  override 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) {
    --- End diff --
    
    Oh wow... Is that because schema is some Seq type that doesn't have a constant time length function?  Should we actually be ensuring the schema is an Array?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] Parquet support for nested ty...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46649042
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46587915
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] Parquet support for nested ty...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46664113
  
    Thanks @rxin for merging! Great to see it finally in master. I understand that it's quite a bit larger than the average PR. Hopefully there won't be too many issues though. Thanks to @marmbrus and @aarondav for reviewing!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40366369
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45437844
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932731
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala ---
    @@ -160,7 +151,7 @@ private[sql] object ParquetRelation {
         }
     
         if (fs.exists(path) &&
    -        !fs.getFileStatus(path)
    +      !fs.getFileStatus(path)
    --- End diff --
    
    I think the indenting was right before.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45416693
  
    Build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45140175
  
    Build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-42779130
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44082041
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45131159
  
    Build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44083405
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15177/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40313975
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14098/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r12061666
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala ---
    @@ -54,9 +54,57 @@ 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) = {
    --- End diff --
    
    Hm, you are right.  That case is probably not handled yet.  However, I think the version here doesn't handle things like `array[1 + 1]`.  So for that reason I think we really need to do all the parsing in the parser.  I'd propose we defer the parsing problem for now.  If we need to do really complex things for unit tests we can hand construct the necessary expressions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11976274
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,582 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +private[parquet] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            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")
    +    }
    +  }
    +
    +  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 PrimitiveRowGroupConverter(attributes)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  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
    +
    +  // Should be only called in root group converter!
    +  def getCurrentRecord: Row
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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!")
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class PrimitiveRowGroupConverter(
    +    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.
    + */
    +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
    +}
    +
    +/**
    + * 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 (complex or primitive)
    + * @param index The position of this (array) field inside its parent converter
    + * @param parent The parent converter
    + * @param buffer A data buffer
    + */
    +private[parquet] class CatalystArrayConverter(
    +    val elementType: DataType,
    +    val index: Int,
    +    protected[parquet] val parent: CatalystConverter,
    +    protected[parquet] var buffer: Buffer[Any])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(elementType: DataType, index: Int, parent: CatalystConverter) =
    +    this(
    +      elementType,
    +      index,
    +      parent,
    +      new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE))
    +
    +  protected[parquet] val converter: Converter = CatalystConverter.createConverter(
    +    new CatalystConverter.FieldType(
    +      CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +      elementType,
    +      false),
    +    fieldIndex=0,
    +    parent=this)
    +
    +  override def getConverter(fieldIndex: Int): Converter = converter
    +
    +  // arrays have only one (repeated) field, which is its elements
    +  override val size = 1
    +
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    // fieldIndex is ignored (assumed to be zero but not checked)
    +    buffer += value
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = {
    +    buffer.clear()
    +  }
    +
    +  override def start(): Unit = {
    +    if (!converter.isPrimitive) {
    +      converter.asInstanceOf[CatalystConverter].clearBuffer
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    assert(parent != null)
    +    parent.updateField(index, new GenericRow(buffer.toArray))
    +    clearBuffer()
    +  }
    +
    +  // Should be only called in root group converter!
    +  override def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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 {
    +
    +  type nativeType = elementType.JvmType
    +
    +  private var buffer: Array[nativeType] = elementType.classTag.newArray(capacity)
    +
    +  private var elements: Int = 0
    +
    +  protected[parquet] val converter: Converter = CatalystConverter.createConverter(
    +    new CatalystConverter.FieldType(
    +      CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +      elementType,
    +      false),
    +    fieldIndex=0,
    +    parent=this)
    +
    +  override def getConverter(fieldIndex: Int): Converter = converter
    +
    +  // arrays have only one (repeated) field, which is its elements
    +  override val size = 1
    +
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit =
    +    throw new UnsupportedOperationException
    +
    +  // 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]
    --- End diff --
    
    Hmmm... this might be a general problem: https://groups.google.com/forum/#!topic/parquet-dev/ueSpQLXtXCk
    
    We need to decide how to handle the case of nullable primitives.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40874845
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-42781102
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44777055
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45416741
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15529/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45439533
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45416740
  
    Build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40313974
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45446576
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15546/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45416599
  
    test this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13711334
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.parquet
    +
    +import java.io.IOException
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.mapreduce.Job
    +
    +import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter}
    +import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData}
    +import parquet.hadoop.util.ContextUtil
    +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType}
    +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._
    +
    +// Implicits
    +import scala.collection.JavaConversions._
    +
    +private[parquet] object ParquetTypesConverter extends Logging {
    +  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]].
    +   *
    +   * We apply the following conversion rules:
    +   * <ul>
    +   *   <li> Primitive types are converter to the corresponding primitive type.</li>
    +   *   <li> Group types that have a single field that is itself a group, which has repetition
    +   *        level `REPEATED`, are treated as follows:<ul>
    +   *          <li> If the nested group has name `values`, the surrounding group is converted
    +   *               into an [[ArrayType]] with the corresponding field type (primitive or
    +   *               complex) as element type.</li>
    +   *          <li> If the nested group has name `map` and two fields (named `key` and `value`),
    +   *               the surrounding group is converted into a [[MapType]]
    +   *               with the corresponding key and value (value possibly complex) types.
    +   *               Note that we currently assume map values are not nullable.</li>
    +   *   <li> Other group types are converted into a [[StructType]] with the corresponding
    +   *        field types.</li></ul></li>
    +   * </ul>
    +   * Note that fields are determined to be `nullable` if and only if their Parquet repetition
    +   * level is not `REQUIRED`.
    +   *
    +   * @param parquetType The type to convert.
    +   * @return The corresponding Catalyst type.
    +   */
    +  def toDataType(parquetType: ParquetType): DataType = {
    +    def correspondsToMap(groupType: ParquetGroupType): Boolean = {
    +      if (groupType.getFieldCount != 1 || groupType.getFields.apply(0).isPrimitive) {
    +        false
    +      } else {
    +        // This mostly follows the convention in ``parquet.schema.ConversionPatterns``
    +        val keyValueGroup = groupType.getFields.apply(0).asGroupType()
    +        keyValueGroup.getRepetition == Repetition.REPEATED &&
    +          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.getFieldName(0) == CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME &&
    +        groupType.getFields.apply(0).getRepetition == Repetition.REPEATED
    +    }
    +
    +    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!
    +          assert(groupType.getFieldCount == 1)
    +          val field = groupType.getFields.apply(0)
    +          new ArrayType(toDataType(field))
    +        }
    +        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))
    +          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 _ => {
    +          // 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))
    +            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))
    +            new ArrayType(elementType)
    +          } else { // everything else: StructType
    +            val fields = groupType
    +              .getFields
    +              .map(ptype => new StructField(
    +              ptype.getName,
    +              toDataType(ptype),
    +              ptype.getRepetition != Repetition.REQUIRED))
    +            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)
    +    // 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
    +  }
    +
    +  /**
    +   * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into
    +   * the corresponding Parquet `Type`.
    +   *
    +   * The conversion follows the rules below:
    +   * <ul>
    +   *   <li> Primitive types are converted into Parquet's primitive types.</li>
    +   *   <li> [[org.apache.spark.sql.catalyst.types.StructType]]s are converted
    +   *        into Parquet's `GroupType` with the corresponding field types.</li>
    +   *   <li> [[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
    +   *        the array as schema. We use Parquet's `ConversionPatterns` for this
    +   *        purpose.</li>
    +   *   <li> [[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` and name `map`. We use Parquet's `ConversionPatterns`
    +   *        for this purpose</li>
    +   * </ul>
    +   * Parquet's repetition level is generally set according to the following rule:
    +   * <ul>
    +   *   <li> If the call to `fromDataType` is recursive inside an enclosing `ArrayType` or
    +   *   `MapType`, then the repetition level is set to `REPEATED`.</li>
    +   *   <li> Otherwise, if the attribute whose type is converted is `nullable`, the Parquet
    +   *   type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.</li>
    +   * </ul>
    +   *
    +   *@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) => {
    +          val parquetElementType = fromDataType(
    +            elementType,
    +            CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +            nullable = false,
    +            inArray = true)
    +            ConversionPatterns.listType(repetition, name, parquetElementType)
    +        }
    +        case StructType(structFields) => {
    +          val fields = structFields.map {
    +            field => fromDataType(field.dataType, field.name, field.nullable, inArray = false)
    +          }
    +          new ParquetGroupType(repetition, name, fields)
    +        }
    +        case MapType(keyType, valueType) => {
    +          val parquetKeyType =
    +            fromDataType(
    +              keyType,
    +              CatalystConverter.MAP_KEY_SCHEMA_NAME,
    +              nullable = false,
    +              inArray = false)
    +          val parquetValueType =
    +            fromDataType(
    +              valueType,
    +              CatalystConverter.MAP_VALUE_SCHEMA_NAME,
    +              nullable = false,
    +              inArray = false)
    +          ConversionPatterns.mapType(
    +            repetition,
    +            name,
    +            parquetKeyType,
    +            parquetValueType)
    +        }
    +        case _ => sys.error(s"Unsupported datatype $ctype")
    +      }
    +    }
    +  }
    +
    +  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 convertFromString(string: String): Seq[Attribute] = {
    +    DataType(string) match {
    +      case s: StructType => s.toAttributes
    +      case other => sys.error(s"Can convert $string to row")
    +    }
    +  }
    +
    +  def convertToString(schema: Seq[Attribute]): String = {
    +    StructType.fromAttributes(schema).toString
    +  }
    +
    +  def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration): Unit = {
    +    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)
    +    extraMetadata.put(
    +      RowReadSupport.SPARK_METADATA_KEY,
    +      ParquetTypesConverter.convertToString(attributes))
    +    // 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(?!)
    --- End diff --
    
    Yeah, a good point. I wonder how one could work around that given that the relation needs to know it's schema when  its execution is being planned in the planner. @marmbrus any ideas?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13661910
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.parquet
    +
    +import java.io.IOException
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.mapreduce.Job
    +
    +import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter}
    +import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData}
    +import parquet.hadoop.util.ContextUtil
    +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType}
    +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._
    +
    +// Implicits
    +import scala.collection.JavaConversions._
    +
    +private[parquet] object ParquetTypesConverter extends Logging {
    +  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]].
    +   *
    +   * We apply the following conversion rules:
    +   * <ul>
    +   *   <li> Primitive types are converter to the corresponding primitive type.</li>
    +   *   <li> Group types that have a single field that is itself a group, which has repetition
    +   *        level `REPEATED`, are treated as follows:<ul>
    +   *          <li> If the nested group has name `values`, the surrounding group is converted
    +   *               into an [[ArrayType]] with the corresponding field type (primitive or
    +   *               complex) as element type.</li>
    +   *          <li> If the nested group has name `map` and two fields (named `key` and `value`),
    +   *               the surrounding group is converted into a [[MapType]]
    +   *               with the corresponding key and value (value possibly complex) types.
    +   *               Note that we currently assume map values are not nullable.</li>
    +   *   <li> Other group types are converted into a [[StructType]] with the corresponding
    +   *        field types.</li></ul></li>
    +   * </ul>
    +   * Note that fields are determined to be `nullable` if and only if their Parquet repetition
    +   * level is not `REQUIRED`.
    +   *
    +   * @param parquetType The type to convert.
    +   * @return The corresponding Catalyst type.
    +   */
    +  def toDataType(parquetType: ParquetType): DataType = {
    +    def correspondsToMap(groupType: ParquetGroupType): Boolean = {
    +      if (groupType.getFieldCount != 1 || groupType.getFields.apply(0).isPrimitive) {
    +        false
    +      } else {
    +        // This mostly follows the convention in ``parquet.schema.ConversionPatterns``
    +        val keyValueGroup = groupType.getFields.apply(0).asGroupType()
    +        keyValueGroup.getRepetition == Repetition.REPEATED &&
    +          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.getFieldName(0) == CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME &&
    +        groupType.getFields.apply(0).getRepetition == Repetition.REPEATED
    +    }
    +
    +    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!
    +          assert(groupType.getFieldCount == 1)
    +          val field = groupType.getFields.apply(0)
    +          new ArrayType(toDataType(field))
    +        }
    +        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))
    +          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 _ => {
    +          // 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))
    +            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))
    +            new ArrayType(elementType)
    +          } else { // everything else: StructType
    +            val fields = groupType
    +              .getFields
    +              .map(ptype => new StructField(
    +              ptype.getName,
    +              toDataType(ptype),
    +              ptype.getRepetition != Repetition.REQUIRED))
    +            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)
    +    // 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
    +  }
    +
    +  /**
    +   * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into
    +   * the corresponding Parquet `Type`.
    +   *
    +   * The conversion follows the rules below:
    +   * <ul>
    +   *   <li> Primitive types are converted into Parquet's primitive types.</li>
    +   *   <li> [[org.apache.spark.sql.catalyst.types.StructType]]s are converted
    +   *        into Parquet's `GroupType` with the corresponding field types.</li>
    +   *   <li> [[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
    +   *        the array as schema. We use Parquet's `ConversionPatterns` for this
    +   *        purpose.</li>
    +   *   <li> [[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` and name `map`. We use Parquet's `ConversionPatterns`
    +   *        for this purpose</li>
    +   * </ul>
    +   * Parquet's repetition level is generally set according to the following rule:
    +   * <ul>
    +   *   <li> If the call to `fromDataType` is recursive inside an enclosing `ArrayType` or
    +   *   `MapType`, then the repetition level is set to `REPEATED`.</li>
    +   *   <li> Otherwise, if the attribute whose type is converted is `nullable`, the Parquet
    +   *   type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.</li>
    +   * </ul>
    +   *
    +   *@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) => {
    +          val parquetElementType = fromDataType(
    +            elementType,
    +            CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +            nullable = false,
    +            inArray = true)
    +            ConversionPatterns.listType(repetition, name, parquetElementType)
    +        }
    +        case StructType(structFields) => {
    +          val fields = structFields.map {
    +            field => fromDataType(field.dataType, field.name, field.nullable, inArray = false)
    +          }
    +          new ParquetGroupType(repetition, name, fields)
    +        }
    +        case MapType(keyType, valueType) => {
    +          val parquetKeyType =
    +            fromDataType(
    +              keyType,
    +              CatalystConverter.MAP_KEY_SCHEMA_NAME,
    +              nullable = false,
    +              inArray = false)
    +          val parquetValueType =
    +            fromDataType(
    +              valueType,
    +              CatalystConverter.MAP_VALUE_SCHEMA_NAME,
    +              nullable = false,
    +              inArray = false)
    +          ConversionPatterns.mapType(
    +            repetition,
    +            name,
    +            parquetKeyType,
    +            parquetValueType)
    +        }
    +        case _ => sys.error(s"Unsupported datatype $ctype")
    +      }
    +    }
    +  }
    +
    +  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 convertFromString(string: String): Seq[Attribute] = {
    +    DataType(string) match {
    +      case s: StructType => s.toAttributes
    +      case other => sys.error(s"Can convert $string to row")
    +    }
    +  }
    +
    +  def convertToString(schema: Seq[Attribute]): String = {
    +    StructType.fromAttributes(schema).toString
    +  }
    +
    +  def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration): Unit = {
    +    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)
    +    extraMetadata.put(
    +      RowReadSupport.SPARK_METADATA_KEY,
    +      ParquetTypesConverter.convertToString(attributes))
    +    // 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(?!)
    --- End diff --
    
    This was not changed in this PR, but this actually makes it so that we can't use S3 credentials set in the SparkContext, or add new FileSystems at runtime, for instance.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13662320
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala ---
    @@ -0,0 +1,409 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.parquet
    +
    +import java.io.IOException
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, Path}
    +import org.apache.hadoop.mapreduce.Job
    +
    +import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter}
    +import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData}
    +import parquet.hadoop.util.ContextUtil
    +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType}
    +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._
    +
    +// Implicits
    +import scala.collection.JavaConversions._
    +
    +private[parquet] object ParquetTypesConverter extends Logging {
    +  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]].
    +   *
    +   * We apply the following conversion rules:
    +   * <ul>
    +   *   <li> Primitive types are converter to the corresponding primitive type.</li>
    +   *   <li> Group types that have a single field that is itself a group, which has repetition
    +   *        level `REPEATED`, are treated as follows:<ul>
    +   *          <li> If the nested group has name `values`, the surrounding group is converted
    +   *               into an [[ArrayType]] with the corresponding field type (primitive or
    +   *               complex) as element type.</li>
    +   *          <li> If the nested group has name `map` and two fields (named `key` and `value`),
    +   *               the surrounding group is converted into a [[MapType]]
    +   *               with the corresponding key and value (value possibly complex) types.
    +   *               Note that we currently assume map values are not nullable.</li>
    +   *   <li> Other group types are converted into a [[StructType]] with the corresponding
    +   *        field types.</li></ul></li>
    +   * </ul>
    +   * Note that fields are determined to be `nullable` if and only if their Parquet repetition
    +   * level is not `REQUIRED`.
    +   *
    +   * @param parquetType The type to convert.
    +   * @return The corresponding Catalyst type.
    +   */
    +  def toDataType(parquetType: ParquetType): DataType = {
    +    def correspondsToMap(groupType: ParquetGroupType): Boolean = {
    +      if (groupType.getFieldCount != 1 || groupType.getFields.apply(0).isPrimitive) {
    +        false
    +      } else {
    +        // This mostly follows the convention in ``parquet.schema.ConversionPatterns``
    +        val keyValueGroup = groupType.getFields.apply(0).asGroupType()
    +        keyValueGroup.getRepetition == Repetition.REPEATED &&
    +          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.getFieldName(0) == CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME &&
    +        groupType.getFields.apply(0).getRepetition == Repetition.REPEATED
    +    }
    +
    +    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!
    +          assert(groupType.getFieldCount == 1)
    +          val field = groupType.getFields.apply(0)
    +          new ArrayType(toDataType(field))
    +        }
    +        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))
    +          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 _ => {
    +          // 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))
    +            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))
    +            new ArrayType(elementType)
    +          } else { // everything else: StructType
    +            val fields = groupType
    +              .getFields
    +              .map(ptype => new StructField(
    +              ptype.getName,
    +              toDataType(ptype),
    +              ptype.getRepetition != Repetition.REQUIRED))
    +            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)
    +    // 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
    +  }
    +
    +  /**
    +   * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into
    +   * the corresponding Parquet `Type`.
    +   *
    +   * The conversion follows the rules below:
    +   * <ul>
    +   *   <li> Primitive types are converted into Parquet's primitive types.</li>
    +   *   <li> [[org.apache.spark.sql.catalyst.types.StructType]]s are converted
    +   *        into Parquet's `GroupType` with the corresponding field types.</li>
    +   *   <li> [[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
    +   *        the array as schema. We use Parquet's `ConversionPatterns` for this
    +   *        purpose.</li>
    +   *   <li> [[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` and name `map`. We use Parquet's `ConversionPatterns`
    +   *        for this purpose</li>
    +   * </ul>
    +   * Parquet's repetition level is generally set according to the following rule:
    +   * <ul>
    +   *   <li> If the call to `fromDataType` is recursive inside an enclosing `ArrayType` or
    +   *   `MapType`, then the repetition level is set to `REPEATED`.</li>
    +   *   <li> Otherwise, if the attribute whose type is converted is `nullable`, the Parquet
    +   *   type gets repetition level `OPTIONAL` and otherwise `REQUIRED`.</li>
    +   * </ul>
    +   *
    +   *@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) => {
    +          val parquetElementType = fromDataType(
    +            elementType,
    +            CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +            nullable = false,
    +            inArray = true)
    +            ConversionPatterns.listType(repetition, name, parquetElementType)
    +        }
    +        case StructType(structFields) => {
    +          val fields = structFields.map {
    +            field => fromDataType(field.dataType, field.name, field.nullable, inArray = false)
    +          }
    +          new ParquetGroupType(repetition, name, fields)
    +        }
    +        case MapType(keyType, valueType) => {
    +          val parquetKeyType =
    +            fromDataType(
    +              keyType,
    +              CatalystConverter.MAP_KEY_SCHEMA_NAME,
    +              nullable = false,
    +              inArray = false)
    +          val parquetValueType =
    +            fromDataType(
    +              valueType,
    +              CatalystConverter.MAP_VALUE_SCHEMA_NAME,
    +              nullable = false,
    +              inArray = false)
    +          ConversionPatterns.mapType(
    +            repetition,
    +            name,
    +            parquetKeyType,
    +            parquetValueType)
    +        }
    +        case _ => sys.error(s"Unsupported datatype $ctype")
    +      }
    +    }
    +  }
    +
    +  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 convertFromString(string: String): Seq[Attribute] = {
    +    DataType(string) match {
    +      case s: StructType => s.toAttributes
    +      case other => sys.error(s"Can convert $string to row")
    +    }
    +  }
    +
    +  def convertToString(schema: Seq[Attribute]): String = {
    +    StructType.fromAttributes(schema).toString
    +  }
    +
    +  def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration): Unit = {
    +    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)
    +    extraMetadata.put(
    +      RowReadSupport.SPARK_METADATA_KEY,
    +      ParquetTypesConverter.convertToString(attributes))
    +    // 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(?!)
    --- End diff --
    
    Good catch, filed as: https://issues.apache.org/jira/browse/SPARK-2112


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-41491324
  
    I added a new issue about the nullability question here: https://issues.apache.org/jira/browse/SPARK-1649


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44081955
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45437841
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45416832
  
    Hey @AndreSchumacher looks like style issues are failing Jenkins.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40302781
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46593586
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40316176
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by aarondav <gi...@git.apache.org>.
Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13892462
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,667 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +/**
    + * Collection of converters of Parquet types (group and primitive types) that
    + * model arrays and maps. The conversions are partly based on the AvroParquet
    + * converters that are part of Parquet in order to be able to process these
    + * types.
    + *
    + * There are several types of converters:
    + * <ul>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive
    + *   (numeric, boolean and String) types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays
    + *   of native JVM element types; note: currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of
    + *   arbitrary element types (including nested element types); note: currently
    + *   null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note:
    + *   currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows
    + *   of only primitive element types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested
    + *   records, including the top-level row record</li>
    + * </ul>
    + */
    +
    +private[sql] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  // 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]
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            fieldIndex,
    +            parent)
    +      }
    +      // 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")
    +    }
    +  }
    +
    +  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)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  /**
    +   * Called by child converters to update their value in its parent (this).
    +   * Note that if possible the more specific update methods below should be used
    +   * to avoid auto-boxing of native JVM types.
    +   *
    +   * @param fieldIndex
    +   * @param value
    +   */
    +  protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit
    +
    +  protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.getBytes)
    +
    +  protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.toStringUsingUTF8)
    +
    +  protected[parquet] def isRootConverter: Boolean = parent == null
    +
    +  protected[parquet] def clearBuffer(): Unit
    +
    +  /**
    +   * Should only be called in the root (group) converter!
    +   *
    +   * @return
    +   */
    +  def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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
    +
    +  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
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class CatalystPrimitiveRowConverter(
    +    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!
    +  override 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) {
    --- End diff --
    
    It's calling something like SeqOptimized.length(), which takes a significant amount of time. We saw around a 33% throughput improvement from this change, on the previously mentioned benchmark.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45130821
  
     Build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40313465
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46612941
  
    @AndreSchumacher do you mind updating the maven build as well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-42779124
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46578605
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40303420
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14095/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40890891
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45131161
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15452/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932114
  
    --- Diff: 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 {
    --- End diff --
    
    Do we need this class?  Arrays don't need to be `Row`s inside of the execution engine, they only need to be of type `Seq`, and even that requirements should probably be removed.  Instead of NativeRow can we just call `toSeq` on the Array?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] Parquet support for nested ty...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46646839
  
    @pwendell good point, forgot about the pom. But now that the avro dependency is removed no changes are neccesary.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40872482
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11987399
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,582 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +private[parquet] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            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")
    +    }
    +  }
    +
    +  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 PrimitiveRowGroupConverter(attributes)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  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
    +
    +  // Should be only called in root group converter!
    +  def getCurrentRecord: Row
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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!")
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class PrimitiveRowGroupConverter(
    +    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.
    + */
    +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
    +}
    +
    +/**
    + * 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 (complex or primitive)
    + * @param index The position of this (array) field inside its parent converter
    + * @param parent The parent converter
    + * @param buffer A data buffer
    + */
    +private[parquet] class CatalystArrayConverter(
    +    val elementType: DataType,
    +    val index: Int,
    +    protected[parquet] val parent: CatalystConverter,
    +    protected[parquet] var buffer: Buffer[Any])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(elementType: DataType, index: Int, parent: CatalystConverter) =
    +    this(
    +      elementType,
    +      index,
    +      parent,
    +      new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE))
    +
    +  protected[parquet] val converter: Converter = CatalystConverter.createConverter(
    +    new CatalystConverter.FieldType(
    +      CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +      elementType,
    +      false),
    +    fieldIndex=0,
    +    parent=this)
    +
    +  override def getConverter(fieldIndex: Int): Converter = converter
    +
    +  // arrays have only one (repeated) field, which is its elements
    +  override val size = 1
    +
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    // fieldIndex is ignored (assumed to be zero but not checked)
    +    buffer += value
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = {
    +    buffer.clear()
    +  }
    +
    +  override def start(): Unit = {
    +    if (!converter.isPrimitive) {
    +      converter.asInstanceOf[CatalystConverter].clearBuffer
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    assert(parent != null)
    +    parent.updateField(index, new GenericRow(buffer.toArray))
    +    clearBuffer()
    +  }
    +
    +  // Should be only called in root group converter!
    +  override def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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 {
    +
    +  type nativeType = elementType.JvmType
    +
    +  private var buffer: Array[nativeType] = elementType.classTag.newArray(capacity)
    +
    +  private var elements: Int = 0
    +
    +  protected[parquet] val converter: Converter = CatalystConverter.createConverter(
    +    new CatalystConverter.FieldType(
    +      CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +      elementType,
    +      false),
    +    fieldIndex=0,
    +    parent=this)
    +
    +  override def getConverter(fieldIndex: Int): Converter = converter
    +
    +  // arrays have only one (repeated) field, which is its elements
    +  override val size = 1
    +
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit =
    +    throw new UnsupportedOperationException
    +
    +  // 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]
    --- End diff --
    
    @marmbrus True, this is a more complicated issue. The discussion here also seems related: https://github.com/Parquet/parquet-mr/issues/158
    
    As also said in the discussion of that Parquet PR I think you can model an array with nullable elements as essentially an array of structs. That would be handled by the other array converter that can contain nested types.
    
    But yeah, the problem is more general. I think it would help to a `nullable` bit inside `DataType` so it's at least clear when something can be null. Now it's only in `StructField`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45444581
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44083404
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13926479
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,667 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +/**
    + * Collection of converters of Parquet types (group and primitive types) that
    + * model arrays and maps. The conversions are partly based on the AvroParquet
    + * converters that are part of Parquet in order to be able to process these
    + * types.
    + *
    + * There are several types of converters:
    + * <ul>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive
    + *   (numeric, boolean and String) types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays
    + *   of native JVM element types; note: currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of
    + *   arbitrary element types (including nested element types); note: currently
    + *   null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note:
    + *   currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows
    + *   of only primitive element types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested
    + *   records, including the top-level row record</li>
    + * </ul>
    + */
    +
    +private[sql] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  // 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]
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            fieldIndex,
    +            parent)
    +      }
    +      // 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")
    +    }
    +  }
    +
    +  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)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] trait CatalystConverter {
    --- End diff --
    
    OK, will do. Great to get some feedback on benchmarks btw. Thanks for doing those.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] WIP Parquet support for neste...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-39861391
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40031248
  
    Hey @AndreSchumacher,  Thanks for working on this.  I don't think we are going to be able to include it for 1.0, but it will be an awesome feature in 1.1.  I will take a detailed look at this as soon as we get all the critical bug fixes in for 1.0.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44081984
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40362749
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44081985
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15176/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932494
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala ---
    @@ -68,14 +93,119 @@ object ParquetTestData {
     
       lazy val testData = new ParquetRelation(testDir.toURI.toString)
     
    +  val testNestedSchema1 =
    +    // 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;
    +        |optional group ownerPhoneNumbers {
    +          |repeated binary array;
    +        |}
    +        |optional group contacts {
    +          |repeated group array {
    +            |required binary name;
    +            |optional binary phoneNumber;
    +          |}
    +        |}
    +      |}
    +    """.stripMargin
    +
    +
    +  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
    +
    +  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
    +
    +  val testNestedSchema4 =
    +    """
    +      |message TestNested4 {
    --- End diff --
    
    Nit: why are the margins aligned with the indentation?  I think that results in an unindented string.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40366371
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14111/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40302375
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46602864
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44082034
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40303294
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45444586
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-42781104
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14889/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46584481
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40892195
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14268/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46593978
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15912/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] WIP Parquet support for neste...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-39861625
  
    @marmbrus I had to do some changes to both the attribute resolution and the SQLParser. Would be great if you could have a look. I think it would be actually much better to parse nested fields together with their datatypes, as it's now done for everything else that is not a simple nested struct.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11986431
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,582 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +private[parquet] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            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")
    +    }
    +  }
    +
    +  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 PrimitiveRowGroupConverter(attributes)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  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
    +
    +  // Should be only called in root group converter!
    +  def getCurrentRecord: Row
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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!")
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class PrimitiveRowGroupConverter(
    +    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.
    + */
    +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
    +}
    +
    +/**
    + * 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 (complex or primitive)
    + * @param index The position of this (array) field inside its parent converter
    + * @param parent The parent converter
    + * @param buffer A data buffer
    + */
    +private[parquet] class CatalystArrayConverter(
    +    val elementType: DataType,
    +    val index: Int,
    +    protected[parquet] val parent: CatalystConverter,
    +    protected[parquet] var buffer: Buffer[Any])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(elementType: DataType, index: Int, parent: CatalystConverter) =
    +    this(
    +      elementType,
    +      index,
    +      parent,
    +      new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE))
    +
    +  protected[parquet] val converter: Converter = CatalystConverter.createConverter(
    +    new CatalystConverter.FieldType(
    +      CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +      elementType,
    +      false),
    +    fieldIndex=0,
    +    parent=this)
    +
    +  override def getConverter(fieldIndex: Int): Converter = converter
    +
    +  // arrays have only one (repeated) field, which is its elements
    +  override val size = 1
    +
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    // fieldIndex is ignored (assumed to be zero but not checked)
    +    buffer += value
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = {
    +    buffer.clear()
    +  }
    +
    +  override def start(): Unit = {
    +    if (!converter.isPrimitive) {
    +      converter.asInstanceOf[CatalystConverter].clearBuffer
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    assert(parent != null)
    +    parent.updateField(index, new GenericRow(buffer.toArray))
    +    clearBuffer()
    +  }
    +
    +  // Should be only called in root group converter!
    +  override def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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 {
    +
    +  type nativeType = elementType.JvmType
    --- End diff --
    
    Good point, thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45130853
  
    Build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45446575
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45140288
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15454/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] Parquet support for nested ty...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46656667
  
    Yay for finally merging this large change in!  Thanks @AndreSchumacher!
    
    BTW, I've been testing this PR on some pretty large / complex schemas for a while so hopefully not too scary merging it at the last minute.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45439536
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15543/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45140285
  
    Build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46584464
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46593975
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40312900
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11986278
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala ---
    @@ -29,25 +31,36 @@ abstract class DataType {
         case e: Expression if e.dataType == this => true
         case _ => false
       }
    +
    +  def isPrimitive(): Boolean = false
     }
     
     case object NullType extends DataType
     
    +trait PrimitiveType extends DataType {
    --- End diff --
    
    @marmbrus `PrimitiveType` is maybe a misnomer. It's the same term that Parquet uses. Basically a `PrimitiveType` is a type that is not contained inside another type (so non-nested). You can argue that a String is a Char array and therefore not primitive but in terms of constructing nested rows it means that a primitive type is a leaf inside the tree that produces a record.
    
    It would help to somehow distinguish between nested and non-nested types. `NativeType` comes close but for example there is `BinaryType` which is primitive but not native.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40303296
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14094/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46587917
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15911/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46593227
  
    That test has been flaky. We are fixing it. 
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] WIP Parquet support for neste...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-39866400
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13893/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] WIP Parquet support for neste...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-39861375
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] Parquet support for nested ty...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46649473
  
    Ok I'm going to merge this in master & branch-1.0 now. Kinda scary but the change is very isolated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40303419
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46612027
  
    @AndreSchumacher do u mind removing the [WIP] tag from the pull request?
    
    Unfortunately due to the avro version bump, we can't include this in 1.0.1. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13926401
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,667 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +/**
    + * Collection of converters of Parquet types (group and primitive types) that
    + * model arrays and maps. The conversions are partly based on the AvroParquet
    + * converters that are part of Parquet in order to be able to process these
    + * types.
    + *
    + * There are several types of converters:
    + * <ul>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive
    + *   (numeric, boolean and String) types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays
    + *   of native JVM element types; note: currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of
    + *   arbitrary element types (including nested element types); note: currently
    + *   null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note:
    + *   currently null values are not supported!</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows
    + *   of only primitive element types</li>
    + *   <li>[[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested
    + *   records, including the top-level row record</li>
    + * </ul>
    + */
    +
    +private[sql] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  // 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]
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            fieldIndex,
    +            parent)
    +      }
    +      // 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")
    +    }
    +  }
    +
    +  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)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  /**
    +   * Called by child converters to update their value in its parent (this).
    +   * Note that if possible the more specific update methods below should be used
    +   * to avoid auto-boxing of native JVM types.
    +   *
    +   * @param fieldIndex
    +   * @param value
    +   */
    +  protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit
    +
    +  protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit =
    +    updateField(fieldIndex, value)
    +
    +  protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.getBytes)
    +
    +  protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit =
    +    updateField(fieldIndex, value.toStringUsingUTF8)
    +
    +  protected[parquet] def isRootConverter: Boolean = parent == null
    +
    +  protected[parquet] def clearBuffer(): Unit
    +
    +  /**
    +   * Should only be called in the root (group) converter!
    +   *
    +   * @return
    +   */
    +  def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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
    +
    +  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
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class CatalystPrimitiveRowConverter(
    +    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!
    +  override 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) {
    --- End diff --
    
    Interesting. OK, will change that to an Array.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44778632
  
    Merged build finished. All automated tests passed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46593240
  
    Jenkins, retest this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-41376843
  
    Yeah, this is probably way more complicated then it needs to be. But part of the code is (I admit premature) optimizations (NativeArrayConverter and MutableRowConverter) so they are kind of optional. Those are based on the discussions we had earlier.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40313462
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40302378
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46593569
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11985972
  
    --- Diff: 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 {
    --- End diff --
    
    @marmbrus  Good question. I think I added that because GetField wants to get a Row when it calls `eval` on its children. I will have another look at that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46589367
  
    @rxin any idea why this one test fails?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44777089
  
    The changes to SqlParser should not have any function effects. I just had to reshuffle a few things to make it easier to extend it to a parser that would support the nested array/map-field expressions, see `NestedSqlParser` in `ParquetQuerySuite`. Currently that parser is incompatible with Hive expressions such as `insert into <database_name>.<table_name>` so we need to revisit that once the syntax is fixed. It's currently there for the tests only.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46644798
  
    That sounds good. If you can just comment that test out for now, that'd be great.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] Parquet support for nested ty...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/360


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46645809
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932640
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala ---
    @@ -0,0 +1,582 @@
    +/*
    + * 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, HashMap}
    +
    +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.parquet.CatalystConverter.FieldType
    +
    +private[parquet] object CatalystConverter {
    +  // The type internally used for fields
    +  type FieldType = StructField
    +
    +  // This is mostly Parquet convention (see, e.g., `ConversionPatterns`).
    +  // Note that "array" for the array elements is chosen by ParquetAvro.
    +  // Using a different value will result in Parquet silently dropping columns.
    +  val ARRAY_ELEMENTS_SCHEMA_NAME = "array"
    +  val MAP_KEY_SCHEMA_NAME = "key"
    +  val MAP_VALUE_SCHEMA_NAME = "value"
    +  val MAP_SCHEMA_NAME = "map"
    +
    +  protected[parquet] def createConverter(
    +      field: FieldType,
    +      fieldIndex: Int,
    +      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)
    +      }
    +      case StructType(fields: Seq[StructField]) => {
    +        new CatalystStructConverter(fields, fieldIndex, parent)
    +      }
    +      case MapType(keyType: DataType, valueType: DataType) => {
    +        new CatalystMapConverter(
    +          Seq(
    +            new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false),
    +            new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)),
    +            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")
    +    }
    +  }
    +
    +  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 PrimitiveRowGroupConverter(attributes)
    +    } else {
    +      new CatalystGroupConverter(attributes)
    +    }
    +  }
    +}
    +
    +private[parquet] 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
    +
    +  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
    +
    +  // Should be only called in root group converter!
    +  def getCurrentRecord: Row
    +}
    +
    +/**
    + * 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.
    + */
    +private[parquet] class CatalystGroupConverter(
    +    protected[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[Row])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(schema: Seq[FieldType], index: Int, parent: CatalystConverter) =
    +    this(
    +      schema,
    +      index,
    +      parent,
    +      current=null,
    +      buffer=new ArrayBuffer[Row](
    +        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!")
    +    // 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)
    +  }
    +
    +  override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex)
    +
    +  // for child converters to update upstream values
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    current.update(fieldIndex, value)
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = buffer.clear()
    +
    +  override def start(): Unit = {
    +    current = ArrayBuffer.fill(schema.length)(null)
    +    converters.foreach {
    +      converter => if (!converter.isPrimitive) {
    +        converter.asInstanceOf[CatalystConverter].clearBuffer
    +      }
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    if (!isRootConverter) {
    +      assert(current!=null) // there should be no empty groups
    +      buffer.append(new GenericRow(current.toArray))
    +      parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]]))
    +    }
    +  }
    +}
    +
    +/**
    + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
    + * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object. Note that his
    + * converter is optimized for rows of primitive types (non-nested records).
    + */
    +private[parquet] class PrimitiveRowGroupConverter(
    +    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.
    + */
    +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
    +}
    +
    +/**
    + * 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 (complex or primitive)
    + * @param index The position of this (array) field inside its parent converter
    + * @param parent The parent converter
    + * @param buffer A data buffer
    + */
    +private[parquet] class CatalystArrayConverter(
    +    val elementType: DataType,
    +    val index: Int,
    +    protected[parquet] val parent: CatalystConverter,
    +    protected[parquet] var buffer: Buffer[Any])
    +  extends GroupConverter with CatalystConverter {
    +
    +  def this(elementType: DataType, index: Int, parent: CatalystConverter) =
    +    this(
    +      elementType,
    +      index,
    +      parent,
    +      new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE))
    +
    +  protected[parquet] val converter: Converter = CatalystConverter.createConverter(
    +    new CatalystConverter.FieldType(
    +      CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME,
    +      elementType,
    +      false),
    +    fieldIndex=0,
    +    parent=this)
    +
    +  override def getConverter(fieldIndex: Int): Converter = converter
    +
    +  // arrays have only one (repeated) field, which is its elements
    +  override val size = 1
    +
    +  override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = {
    +    // fieldIndex is ignored (assumed to be zero but not checked)
    +    buffer += value
    +  }
    +
    +  override protected[parquet] def clearBuffer(): Unit = {
    +    buffer.clear()
    +  }
    +
    +  override def start(): Unit = {
    +    if (!converter.isPrimitive) {
    +      converter.asInstanceOf[CatalystConverter].clearBuffer
    +    }
    +  }
    +
    +  override def end(): Unit = {
    +    assert(parent != null)
    +    parent.updateField(index, new GenericRow(buffer.toArray))
    +    clearBuffer()
    +  }
    +
    +  // Should be only called in root group converter!
    +  override def getCurrentRecord: Row = throw new UnsupportedOperationException
    +}
    +
    +/**
    + * 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 {
    +
    +  type nativeType = elementType.JvmType
    --- End diff --
    
    Capital letters for Types.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40892194
  
    Merged build finished. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-45416688
  
     Build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40890889
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932023
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala ---
    @@ -54,9 +54,57 @@ 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) = {
    --- End diff --
    
    What do you think of #518 instead of changing the resolver?  I am not a parser expert, but I think this is closer to the way hive (and probably optiq which we hope to use eventually) work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] Parquet support for nested ty...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46649043
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15939/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44081960
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r13373005
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala ---
    @@ -29,25 +31,36 @@ abstract class DataType {
         case e: Expression if e.dataType == this => true
         case _ => false
       }
    +
    +  def isPrimitive(): Boolean = false
    --- End diff --
    
    No `()`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40312907
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by marmbrus <gi...@git.apache.org>.
Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11932124
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala ---
    @@ -52,6 +52,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression {
           }
         } else {
           val baseValue = child.eval(input).asInstanceOf[Map[Any, _]]
    +      // TODO: recover key type!!
    --- End diff --
    
    What do you mean?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11985586
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala ---
    @@ -54,9 +54,57 @@ 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) = {
    --- End diff --
    
    Thanks, I will have a look. One question: does this also handle maps and nested fields inside arrays, like `struct.array1[1].field1.map1["key1"].array2[0]`? I don't know Optiq but I will still check out how Hive does this. Since there was (is?) no support for nested Parquet types in Hive that may be a dead end though.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46602867
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15914/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40362755
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40238811
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40872477
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44778633
  
    All automated tests passed.
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/15328/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-44777060
  
    Merged build started. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46578586
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-46644749
  
    @rxin the avro dependency is for the tests only (to make sure we can read parquet files with avro objects in them). I can remove the one test if that blocks it from being included. When the rest of the build has caught up with the version we can add it again. What do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40238795
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40302779
  
     Merged build triggered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AndreSchumacher <gi...@git.apache.org>.
Github user AndreSchumacher commented on a diff in the pull request:

    https://github.com/apache/spark/pull/360#discussion_r11989530
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala ---
    @@ -153,9 +153,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")
    --- End diff --
    
    @marmbrus Good question. I'm not yet totally sure myself. But consider the following example: You have an array of structs, which have another array as field. So something like:
    
    `ArrayType(StructType(Seq(ArrayType(IntegerType))))`
    
    Lets call the inner array `inner` and the outer array `outer`. Note that `outer` could be itself just a field in a higher-level record.
    
    Now whenever Parquet is done passing the data for the current `inner` it will let you know by calling `end` on the converter for that field, in this case an array converter. Now the current struct has been processed completely, so its converter's `end` will be called, too. The current `outer` record, however, may or not may be completed. If it's not completed, then the current `inner` needs to be stored somewhere and you cannot use a mutable row because it is not yet save to reuse that chunk of memory whenever the next `inner` comes along.
    
    Does this make any sense at all? I'm happy to discuss other solutions, too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: SPARK-1293 [SQL] [WIP] Parquet support for nes...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/360#issuecomment-40874847
  
    
    Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14256/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---