Github user julienledem commented on a diff in the pull request: https://github.com/apache/spark/pull/15821#discussion_r105322283 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala --- @@ -0,0 +1,411 @@ +/* +* 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.ByteBuffer +import java.nio.channels.{Channels, SeekableByteChannel} + +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.{ArrowReader, 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._ + + +/** + * ArrowReader requires a seekable byte channel. + * NOTE - this is taken from test org.apache.vector.file, see about moving to public util pkg + */ +private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: Array[Byte]) + extends SeekableByteChannel { + var _position: Long = 0L + + override def isOpen: Boolean = { + byteArray != null + } + + override def close(): Unit = { + byteArray = null + } + + override def read(dst: ByteBuffer): Int = { + val remainingBuf = byteArray.length - _position + val length = Math.min(dst.remaining(), remainingBuf).toInt + dst.put(byteArray, _position.toInt, length) + _position += length + length.toInt + } + + override def position(): Long = _position + + override def position(newPosition: Long): SeekableByteChannel = { + _position = newPosition.toLong + this + } + + override def size: Long = { + byteArray.length.toLong + } + + override def write(src: ByteBuffer): Int = { + throw new UnsupportedOperationException("Read Only") + } + + override def truncate(size: Long): SeekableByteChannel = { + throw new UnsupportedOperationException("Read Only") + } +} + +/** + * Intermediate data structure returned from Arrow conversions + */ +private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch] + +/** + * Build a payload from existing ArrowRecordBatches + */ +private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends ArrowPayload { + private val iter = batches.iterator + override def next(): ArrowRecordBatch = iter.next() + override def hasNext: Boolean = iter.hasNext +} + +/** + * 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 + + def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: StructType): ArrowPayload = { + val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, schema, allocator) + new ArrowStaticPayload(batch) + } + + def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): ArrowPayload = { + val batches = scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch] + var i = 0 + while (i < payloadByteArrays.length) { + val payloadBytes = payloadByteArrays(i) + val in = new ByteArrayReadableSeekableByteChannel(payloadBytes) + val reader = new ArrowReader(in, _allocator) + val footer = reader.readFooter() + val batchBlocks = footer.getRecordBatches.asScala.toArray + batchBlocks.foreach(block => batches += reader.readRecordBatch(block)) + i += 1 + } + new ArrowStaticPayload(batches: _*) + } +} + +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") + } + } + + /** + * Iterate over InternalRows and write to an ArrowRecordBatch. + */ + private def internalRowIterToArrowBatch( + rowIter: Iterator[InternalRow], + schema: StructType, + allocator: RootAllocator): 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 fieldAndBuf = columnWriters.map { writer => + writer.finish() + }.unzip + val fieldNodes = fieldAndBuf._1 + val buffers = fieldAndBuf._2.flatten --- End diff -- how about ``` val (fieldNodes, buf) = columnWriters.map( _.finish()).unzip val buffers = buf.flatten ```
--- 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