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

    https://github.com/apache/spark/pull/15821#discussion_r113730387
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
    @@ -0,0 +1,396 @@
    +/*
    +* 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.execution.arrow
    +
    +import java.io.ByteArrayOutputStream
    +import java.nio.channels.Channels
    +
    +import scala.collection.JavaConverters._
    +
    +import io.netty.buffer.ArrowBuf
    +import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
    +import org.apache.arrow.vector._
    +import org.apache.arrow.vector.BaseValueVector.BaseMutator
    +import org.apache.arrow.vector.file._
    +import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
    +import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
    +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
    +import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
    +
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.Utils
    +
    +
    +/**
    + * Store Arrow data in a form that can be serialized by Spark
    + */
    +private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
    +
    +  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
    +    this(ArrowConverters.batchToByteArray(batch, schema, allocator))
    +  }
    +
    +  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
    +    ArrowConverters.byteArrayToBatch(batchBytes, allocator)
    +  }
    +}
    +
    +private[sql] object ArrowConverters {
    +
    +  /**
    +   * Map a Spark DataType to ArrowType.
    +   */
    +  private[arrow] 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 _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
    +    }
    +  }
    +
    +  /**
    +   * Convert a Spark Dataset schema to Arrow schema.
    +   */
    +  private[arrow] 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)
    +  }
    +
    +  /**
    +   * Maps Iterator from InternalRow to ArrowPayload
    +   */
    +  private[sql] def toPayloadIterator(
    +      rowIter: Iterator[InternalRow],
    +      schema: StructType): Iterator[ArrowPayload] = {
    +    new Iterator[ArrowPayload] {
    +      private val _allocator = new RootAllocator(Long.MaxValue)
    +      private var _nextPayload = if (rowIter.nonEmpty) convert() else null
    +
    +      override def hasNext: Boolean = _nextPayload != null
    +
    +      override def next(): ArrowPayload = {
    +        val obj = _nextPayload
    +        if (hasNext) {
    +          if (rowIter.hasNext) {
    +            _nextPayload = convert()
    +          } else {
    +            _allocator.close()
    +            _nextPayload = null
    +          }
    +        }
    +        obj
    +      }
    +
    +      private def convert(): ArrowPayload = {
    +        val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator)
    +        new ArrowPayload(batch, schema, _allocator)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Iterate over InternalRows and write to an ArrowRecordBatch.
    +   */
    +  private def internalRowIterToArrowBatch(
    +      rowIter: Iterator[InternalRow],
    +      schema: StructType,
    +      allocator: BufferAllocator): ArrowRecordBatch = {
    +
    +    val columnWriters = schema.fields.zipWithIndex.map { case (field, 
ordinal) =>
    +      ColumnWriter(ordinal, allocator, field.dataType).init()
    +    }
    +
    +    val writerLength = columnWriters.length
    +    while (rowIter.hasNext) {
    +      val row = rowIter.next()
    +      var i = 0
    +      while (i < writerLength) {
    +        columnWriters(i).write(row)
    +        i += 1
    +      }
    +    }
    +
    +    val (fieldNodes, bufferArrays) = columnWriters.map(_.finish()).unzip
    +    val buffers = bufferArrays.flatten
    +
    +    val rowLength = if (fieldNodes.nonEmpty) fieldNodes.head.getLength 
else 0
    +    val recordBatch = new ArrowRecordBatch(rowLength,
    +      fieldNodes.toList.asJava, buffers.toList.asJava)
    +
    +    buffers.foreach(_.release())
    +    recordBatch
    +  }
    +
    +  /**
    +   * Convert an ArrowRecordBatch to a byte array and close batch
    +   */
    +  private[arrow] def batchToByteArray(
    --- End diff --
    
    Maybe useful to document that the `batch` becomes invalid after calling 
this function and should not be used further.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to