Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 090AD200C63 for ; Thu, 27 Apr 2017 03:45:38 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 079DA160BB4; Thu, 27 Apr 2017 01:45:38 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 281B1160BA8 for ; Thu, 27 Apr 2017 03:45:37 +0200 (CEST) Received: (qmail 35964 invoked by uid 500); 27 Apr 2017 01:45:36 -0000 Mailing-List: contact reviews-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list reviews@spark.apache.org Received: (qmail 35953 invoked by uid 99); 27 Apr 2017 01:45:36 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 27 Apr 2017 01:45:36 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id EBC5EE1103; Thu, 27 Apr 2017 01:45:35 +0000 (UTC) From: BryanCutler To: reviews@spark.apache.org Reply-To: reviews@spark.apache.org References: In-Reply-To: Subject: [GitHub] spark pull request #15821: [SPARK-13534][PySpark] Using Apache Arrow to incr... Content-Type: text/plain Message-Id: <20170427014535.EBC5EE1103@git1-us-west.apache.org> Date: Thu, 27 Apr 2017 01:45:35 +0000 (UTC) archived-at: Thu, 27 Apr 2017 01:45:38 -0000 Github user BryanCutler commented on a diff in the pull request: https://github.com/apache/spark/pull/15821#discussion_r113600189 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala --- @@ -0,0 +1,432 @@ +/* +* 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._ +import org.apache.spark.util.Utils + + +/** + * ArrowReader requires a seekable byte channel. + * TODO: This is available in arrow-vector now with ARROW-615, to be included in 0.2.1 release + */ +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 + } + + 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 + + /** + * Iterate over the rows and convert to an ArrowPayload, using RootAllocator from this class + */ + def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: StructType): ArrowPayload = { + val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, schema, _allocator) + new ArrowStaticPayload(batch) + } + + /** + * Read an Array of Arrow Record batches as byte Arrays into an ArrowPayload, using + * RootAllocator from this class + */ + 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: _*) + } + + /** + * Call when done using this converter, will close RootAllocator so any ArrowBuffers should be + * closed first + */ + def close(): Unit = { + _allocator.close() + } +} + +private[sql] object ArrowConverters { + + /** + * Map a Spark Dataset type to ArrowType. + */ + private[sql] def sparkTypeToArrowType(dataType: DataType): ArrowType = { --- End diff -- My latest update needed to use this outside of the object, but I will change it to `private[arrow]` --- 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