You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by viirya <gi...@git.apache.org> on 2017/02/15 07:13:52 UTC

[GitHub] spark pull request #15821: [SPARK-13534][WIP][PySpark] Using Apache Arrow to...

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

    https://github.com/apache/spark/pull/15821#discussion_r101214262
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
    @@ -0,0 +1,360 @@
    +/*
    +* 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
    +
    +import java.io.ByteArrayOutputStream
    +import java.nio.channels.Channels
    +
    +import scala.collection.JavaConverters._
    +
    +import io.netty.buffer.ArrowBuf
    +import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
    +import org.apache.arrow.vector._
    +import org.apache.arrow.vector.BaseValueVector.BaseMutator
    +import org.apache.arrow.vector.file.ArrowWriter
    +import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
    +import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
    +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
    +
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Intermediate data structure returned from Arrow conversions
    + */
    +private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
    +
    +/**
    + * Class that wraps an Arrow RootAllocator used in conversion
    + */
    +private[sql] class ArrowConverters {
    +  private val _allocator = new RootAllocator(Long.MaxValue)
    +
    +  private[sql] def allocator: RootAllocator = _allocator
    +
    +  private class ArrowStaticPayload(batches: ArrowRecordBatch*) extends ArrowPayload {
    +    private val iter = batches.iterator
    +
    +    override def next(): ArrowRecordBatch = iter.next()
    +    override def hasNext: Boolean = iter.hasNext
    +  }
    +
    +  def internalRowsToPayload(rows: Array[InternalRow], schema: StructType): ArrowPayload = {
    +    val batch = ArrowConverters.internalRowsToArrowRecordBatch(rows, schema, allocator)
    +    new ArrowStaticPayload(batch)
    +  }
    +}
    +
    +private[sql] object ArrowConverters {
    +
    +  /**
    +   * Map a Spark Dataset type to ArrowType.
    +   */
    +  private[sql] def sparkTypeToArrowType(dataType: DataType): ArrowType = {
    +    dataType match {
    +      case BooleanType => ArrowType.Bool.INSTANCE
    +      case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
    +      case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, true)
    +      case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
    +      case FloatType => new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
    +      case DoubleType => new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
    +      case ByteType => new ArrowType.Int(8, true)
    +      case StringType => ArrowType.Utf8.INSTANCE
    +      case BinaryType => ArrowType.Binary.INSTANCE
    +      case DateType => ArrowType.Date.INSTANCE
    +      case TimestampType => new ArrowType.Timestamp(TimeUnit.MILLISECOND)
    +      case _ => throw new UnsupportedOperationException(s"Unsupported data type: $dataType")
    +    }
    +  }
    +
    +  /**
    +   * Transfer an array of InternalRow to an ArrowRecordBatch.
    +   */
    +  private[sql] def internalRowsToArrowRecordBatch(
    +      rows: Array[InternalRow],
    +      schema: StructType,
    +      allocator: RootAllocator): ArrowRecordBatch = {
    +    val fieldAndBuf = schema.fields.zipWithIndex.map { case (field, ordinal) =>
    +      internalRowToArrowBuf(rows, ordinal, field, allocator)
    +    }.unzip
    +    val fieldNodes = fieldAndBuf._1.flatten
    +    val buffers = fieldAndBuf._2.flatten
    +
    +    val recordBatch = new ArrowRecordBatch(rows.length,
    +      fieldNodes.toList.asJava, buffers.toList.asJava)
    +
    +    buffers.foreach(_.release())
    +    recordBatch
    +  }
    +
    +  /**
    +   * Write a Field from array of InternalRow to an ArrowBuf.
    +   */
    +  private def internalRowToArrowBuf(
    +      rows: Array[InternalRow],
    +      ordinal: Int,
    +      field: StructField,
    +      allocator: RootAllocator): (Array[ArrowFieldNode], Array[ArrowBuf]) = {
    +    val numOfRows = rows.length
    +    val columnWriter = ColumnWriter(allocator, field.dataType)
    +    columnWriter.init(numOfRows)
    +    var index = 0
    +
    +    while(index < numOfRows) {
    +      val row = rows(index)
    +      if (row.isNullAt(ordinal)) {
    +        columnWriter.writeNull()
    +      } else {
    +        columnWriter.write(row, ordinal)
    +      }
    +      index += 1
    +    }
    +
    +    val (arrowFieldNodes, arrowBufs) = columnWriter.finish()
    +    (arrowFieldNodes.toArray, arrowBufs.toArray)
    +  }
    +
    +  /**
    +   * Convert a Spark Dataset schema to Arrow schema.
    +   */
    +  private[sql] def schemaToArrowSchema(schema: StructType): Schema = {
    +    val arrowFields = schema.fields.map { f =>
    +      new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), List.empty[Field].asJava)
    +    }
    +    new Schema(arrowFields.toList.asJava)
    +  }
    +
    +  /**
    +   * Write an ArrowPayload to a byte array
    +   */
    +  private[sql] def payloadToByteArray(payload: ArrowPayload, schema: StructType): Array[Byte] = {
    +    val arrowSchema = ArrowConverters.schemaToArrowSchema(schema)
    +    val out = new ByteArrayOutputStream()
    +    val writer = new ArrowWriter(Channels.newChannel(out), arrowSchema)
    +    try {
    +      payload.foreach(writer.writeRecordBatch)
    +    } catch {
    +      case e: Exception =>
    +        throw e
    +    } finally {
    +      writer.close()
    +      payload.foreach(_.close())
    +    }
    +    out.toByteArray
    +  }
    +}
    +
    +private[sql] trait ColumnWriter {
    +  def init(initialSize: Int): Unit
    +  def writeNull(): Unit
    +  def write(row: InternalRow, ordinal: Int): Unit
    +
    +  /**
    +   * Clear the column writer and return the ArrowFieldNode and ArrowBuf.
    +   * This should be called only once after all the data is written.
    +   */
    +  def finish(): (Seq[ArrowFieldNode], Seq[ArrowBuf])
    +}
    +
    +/**
    + * Base class for flat arrow column writer, i.e., column without children.
    + */
    +private[sql] abstract class PrimitiveColumnWriter(protected val allocator: BaseAllocator)
    +    extends ColumnWriter {
    +  protected def valueVector: BaseDataValueVector
    +  protected def valueMutator: BaseMutator
    +
    +  protected def setNull(): Unit
    +  protected def setValue(row: InternalRow, ordinal: Int): Unit
    +
    +  protected var count = 0
    +  protected var nullCount = 0
    +
    +  override def init(initialSize: Int): Unit = {
    +    valueVector.allocateNew()
    +  }
    +
    +  override def writeNull(): Unit = {
    +    setNull()
    +    nullCount += 1
    +    count += 1
    +  }
    +
    +  override def write(row: InternalRow, ordinal: Int): Unit = {
    +    setValue(row, ordinal)
    +    count += 1
    +  }
    +
    +  override def finish(): (Seq[ArrowFieldNode], Seq[ArrowBuf]) = {
    +    valueMutator.setValueCount(count)
    +    val fieldNode = new ArrowFieldNode(count, nullCount)
    +    val valueBuffers: Seq[ArrowBuf] = valueVector.getBuffers(true)
    +    (List(fieldNode), valueBuffers)
    +  }
    +}
    +
    +private[sql] class BooleanColumnWriter(allocator: BaseAllocator)
    +    extends PrimitiveColumnWriter(allocator) {
    +  private def bool2int(b: Boolean): Int = if (b) 1 else 0
    +
    +  override protected val valueVector: NullableBitVector
    +    = new NullableBitVector("BooleanValue", allocator)
    +  override protected val valueMutator: NullableBitVector#Mutator = valueVector.getMutator
    +
    +  override def setNull(): Unit = valueMutator.setNull(count)
    --- End diff --
    
    setNull and setValue should be protected.


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

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org