You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by BryanCutler <gi...@git.apache.org> on 2018/06/12 21:33:20 UTC

[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

GitHub user BryanCutler opened a pull request:

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

    [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format for creating from and collecting Pandas DataFrames

    ## What changes were proposed in this pull request?
    
    This changes the calls of `toPandas()` and `createDataFrame()` to use the Arrow stream format, when Arrow is enabled.  Previously, Arrow data was written to byte arrays where each chunk is an output of the Arrow file format.  This was mainly due to constraints at the time, and caused some overhead by writing the schema/footer on each chunk of data and then having to read multiple Arrow file inputs and concat them together.
    
    Using the Arrow stream format has improved these by increasing performance, lower memory overhead for the average case, and simplified the code.  Here are the details of this change:
    
    **toPandas()**
    
    _Before:_
    Spark internal rows are converted to Arrow file format, each group of records is a complete Arrow file which contains the schema and other metadata.  Next a collect is done and an Array of Arrow files is the result.  After that each Arrow file is sent to Python driver which then loads each file and concats them to a single Arrow DataFrame.
    
    _After:_
    Spark internal rows are converted to ArrowRecordBatches directly, which is the simplest Arrow component for IPC data transfers.  The driver JVM then immediately starts serving data to Python as an Arrow stream, sending the schema first. It then starts Spark jobs with a custom handler such that when a partition is received (and in the correct order) the ArrowRecordBatches can be sent to python as soon as possible.  This improves performance, simplifies memory usage on executors, and improves the average memory usage on the JVM driver.  Since the order of partitions must be preserved, the worst case is that the first partition will be the last to arrive and all data must be kept in memory until then.  This case is no worse that before when doing a full collect.
    
    **createDataFrame()**
    
    _Before:_
    A Pandas DataFrame is split into parts and each part is made into an Arrow file.  Then each file is prefixed by the buffer size and written to a temp file.  The temp file is read and each Arrow file is parallelized as a byte array.
    
    _After:_
    A Pandas DataFrame is split into parts, then an Arrow stream is written to a temp file where each part is an ArrowRecordBatch.  The temp file is read as a stream and the Arrow messages are examined.  If the message is an ArrowRecordBatch, the data is saved as a byte array.  After reading the file, each ArrowRecordBatch is parallelized as a byte array.  This has slightly more processing than before because we must look each Arrow message to extract the record batches, but performance remains the same.  It is cleaner in the sense that IPC from Python to JVM is done over a single Arrow stream.
    
    ## How was this patch tested?
    
    Added new unit tests for the additions to ArrowConverters in Scala, existing tests for Python.

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

    $ git pull https://github.com/BryanCutler/spark arrow-toPandas-stream-SPARK-23030

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

    https://github.com/apache/spark/pull/21546.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 #21546
    
----
commit 9af482170ee95831bbda139e6e931ba2631df386
Author: Bryan Cutler <cu...@...>
Date:   2018-01-10T22:02:15Z

    change ArrowConverters to stream format

commit d617f0da8eff1509da465bb707340e391314bec4
Author: Bryan Cutler <cu...@...>
Date:   2018-01-10T22:14:07Z

    Change ArrowSerializer to use stream format

commit f10d5d9cd3cece7f56749e1de7fe01699e4759a0
Author: Bryan Cutler <cu...@...>
Date:   2018-01-12T00:40:36Z

    toPandas is working with RecordBatch payloads, using custom handler to stream ordered partitions

commit 03653c687473b82bbfb6653504479498a2a3c63b
Author: Bryan Cutler <cu...@...>
Date:   2018-02-10T00:23:17Z

    cleanup and removed ArrowPayload, createDataFrame now working

commit 1b932463bca0815e79f3a8d61d1c816e62949698
Author: Bryan Cutler <cu...@...>
Date:   2018-03-09T00:14:06Z

    toPandas and createDataFrame working but tests fail with date cols

commit ce22d8ad18e052d150528752b727c6cfe11485f7
Author: Bryan Cutler <cu...@...>
Date:   2018-03-27T00:32:03Z

    removed usage of seekableByteChannel

commit dede0bd96921c439747a9176f24c9ecbb9c8ce0a
Author: Bryan Cutler <cu...@...>
Date:   2018-03-28T00:28:54Z

    for toPandas, set old collection result to null and add comments

commit 9e29b092cb7d45fa486db0215c3bd4a99c5f8d98
Author: Bryan Cutler <cu...@...>
Date:   2018-03-28T18:28:18Z

    cleanup, not yet passing ArrowConvertersSuite

commit ceb8d38a6c83c3b6dae040c9e8d860811ecad0cc
Author: Bryan Cutler <cu...@...>
Date:   2018-03-29T21:14:03Z

    fix to read Arrow stream with multiple batches, cleanup, add docs, scala tests pass, style pass

commit f42e4ea7b4fb944eeefd39a0fd6a1428b527214a
Author: Bryan Cutler <cu...@...>
Date:   2018-03-29T22:17:25Z

    use base OutputStream for serveToStream instead of DataOutputStream

commit 951843d760aa6d29ff18112e82d28f4f6dc09907
Author: Bryan Cutler <cu...@...>
Date:   2018-03-29T22:21:13Z

    accidentally removed date type checks, passing pyspark tests

commit af03c6b384fe4ea73d67ad1d3f46be4a1e027e9e
Author: Bryan Cutler <cu...@...>
Date:   2018-06-12T00:29:12Z

    Changed to only use Arrow batch bytes as payload, had to hack Arrow MessageChannelReader

commit b047c1624429ea579aa279e92909b90400b40c58
Author: Bryan Cutler <cu...@...>
Date:   2018-06-12T00:30:48Z

    added todo comment

commit a77b89ea0357e3ce146ff35537eb7da8a8c80bad
Author: Bryan Cutler <cu...@...>
Date:   2018-06-12T17:35:05Z

    change getBatchesFromStream to return iterator

commit 81c82093edf78d36a1de850b3d8faede88fb0524
Author: Bryan Cutler <cu...@...>
Date:   2018-06-12T18:06:22Z

    need to end stream on toPandas after all batches sent to python, and added some comments

commit 5f46a02aa34e9f51ac310d27e3272b883c67cc37
Author: Bryan Cutler <cu...@...>
Date:   2018-06-12T18:11:19Z

    forgot to remove old comment

commit 7694b8fe6970789d4e88f9c5df46c97a4b235f02
Author: Bryan Cutler <cu...@...>
Date:   2018-06-12T18:47:49Z

    fixed up comments

commit a5a1fbe7121c5b0dd93876a56c29ad17dcd9b168
Author: Bryan Cutler <cu...@...>
Date:   2018-06-12T20:22:30Z

    fixed up some wording

----


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212158131
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -111,65 +113,58 @@ private[sql] object ArrowConverters {
                 rowCount += 1
               }
               arrowWriter.finish()
    -          writer.writeBatch()
    +          val batch = unloader.getRecordBatch()
    +          MessageSerializer.serialize(writeChannel, batch)
    +          batch.close()
    --- End diff --
    
    Should we `tryWithResouce` here too?


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195562446
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    I wonder if this works?
    In a for-loop:
    (1) Read next batch into VectorSchemaRoot (copy into arrow memory)
    (2) Use VectorUnloader to unload the VectorSchemaRoot to an ArrowRecordBatch (no copy)
    (3) Use MessageSerializer.serialize to write ArrowRecordBatch to a ByteChannel (copy from arrow memory to java memory)
    
    Seems that we cannot directly read from socket into java memory anyway (have to go through Arrow memory allocator)..


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204479024
  
    --- Diff: python/pyspark/sql/dataframe.py ---
    @@ -2095,9 +2095,11 @@ def toPandas(self):
                             _check_dataframe_localize_timestamps
                         import pyarrow
     
    -                    tables = self._collectAsArrow()
    -                    if tables:
    -                        table = pyarrow.concat_tables(tables)
    +                    # Collect un-ordered list of batches, and list of correct order indices
    +                    batches, batch_order = self._collectAsArrow()
    +                    if batches:
    --- End diff --
    
    Sure, I was playing around with this being an iterator, but I will change it since it is a list now


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212163122
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +178,106 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeRecordBatch(
    +      new ReadChannel(Channels.newChannel(in)), allocator)  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    --- End diff --
    
    @BryanCutler, why did this parallelize with the length of batches size? I thought the data size is usually small and wondering if it necessarily speeds up in general. Did I misread?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95390 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95390/testReport)** for PR 21546 at commit [`ffb47cb`](https://github.com/apache/spark/commit/ffb47cb2d411b91e240ab40cd6bd75b025e417c2).


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r197538429
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    Ok...  I don't really need to subclass `MessageChannelReader` here.  What if instead we just make a couple of static functions on the Arrow side to help with the details of processing messages, like:
    
    ```Java
    public class MessageChannelReader {
    ...
    
      public static Integer readMessageLength(ReadChannel in) {..}
    
      public static Message loadMessage(ReadChannel in, int messageLength, ByteBuffer buffer) {..}
    }
    ```
    Is that better?



---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92487/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95363/
    Test FAILed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95310 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95310/testReport)** for PR 21546 at commit [`2fe46f8`](https://github.com/apache/spark/commit/2fe46f82dc38af972bc0974aca1fd846bcb483e5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199476976
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val messageLength = readMessageLength(inputChannel)
    +        if (messageLength == 0) {
    +          return null
    +        }
    +
    +        val buffer = ByteBuffer.allocate(messageLength)
    +        val msg = loadMessage(inputChannel, messageLength, buffer)
    +        val bodyLength = msg.bodyLength().asInstanceOf[Int]
    --- End diff --
    
    why not `toInt`?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91733/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/628/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95390/
    Test FAILed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95397/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2578/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    ### Performance Tests - toPandas
    
    Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
    measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.
    
    Test code
    ```python
    df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
    for i in range(5):
    	start = time.time()
    	_ = df.toPandas()
    	elapsed = time.time() - start
    ```
    
    | Current Master | This PR |
    |---------------------|------------|
    5.803557 | 4.342533
    5.409119 | 4.399408
    5.493509 | 4.468471
    5.433107 | 4.36524
    5.488757 | 4.373791
    
    | Avg Master | Avg This PR
    -----------|--------------
    5.5256098 | 4.3898886 
    
    #### Speedup of 1.258712989
    
    
    
    



---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r197932884
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    --- End diff --
    
    Looking at this again, `it.toArray` is run on the executor, which ends up doing the same thing as `collect()` and then `handlePartitions` is run on the results of that in the driver.  The task results need to be serialized, so I'm not sure if we can avoid `it.toArray` here, any thoughts @ueshin ?
    



---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Hey @BryanCutler, btw, mind i fI ask move the benchmarks into the PR description?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95325/
    Test FAILed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Hey @HyukjinKwon , after going through the previous benchmarks, it seems out-of-order batches had more of an effect on performance that I thought with `toPandas`. The current revision of this PR (which buffers out of order batches in the driver JVM) has about a 1.06x - 1.09x speedup which seems a bit underwhelming after getting ~1.25x when sending out-of-order batches. I still want to try to verify the old numbers and will hopefully get to that tomorrow.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95145 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95145/testReport)** for PR 21546 at commit [`5549644`](https://github.com/apache/spark/commit/554964465dbcb99cc313620fafb0fc41acfd4304).
     * This patch passes all tests.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r198000610
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    --- End diff --
    
    Oh, I see. In that case, we need to do `it.toArray`. Thanks.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199612847
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala ---
    @@ -34,17 +33,19 @@ private[sql] object PythonSQLUtils {
       }
     
       /**
    -   * Python Callable function to convert ArrowPayloads into a [[DataFrame]].
    +   * Python callable function to read a file in Arrow stream format and create a [[DataFrame]]
    +   * using each serialized ArrowRecordBatch as a partition.
        *
    -   * @param payloadRDD A JavaRDD of ArrowPayloads.
    -   * @param schemaString JSON Formatted Schema for ArrowPayloads.
        * @param sqlContext The active [[SQLContext]].
    -   * @return The converted [[DataFrame]].
    +   * @param filename File to read the Arrow stream from.
    +   * @param schemaString JSON Formatted Spark schema for Arrow batches.
    +   * @return A new [[DataFrame]].
        */
    -  def arrowPayloadToDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    -      schemaString: String,
    -      sqlContext: SQLContext): DataFrame = {
    -    ArrowConverters.toDataFrame(payloadRDD, schemaString, sqlContext)
    +  def arrowReadStreamFromFile(
    --- End diff --
    
    `arrowStreamFromFile` is important to get in the name since it is a stream format being read from a file, but how about `arrowStreamFromFileToDataFrame`?  Its a bit long but it would be good to indicate that it produces a `DataFrame` for the call from Python.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r196169993
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    >The whole class here is trying to read Arrow record batches from an stream into Java on-heap memory without going through Arrow off-heap memory, is that correct?
    
    Yes, that's correct.  This is done to parallelize the Arrow record batches.
    
    > Also, this function is only used for pandas DataFrame -> Spark DataFrame?
    
    Yes, `RecordBatchMessageReader` is a specialized class and only meant for this purpose.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199623502
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    --- End diff --
    
    I don't think there is any reason since this is just internal and it's an anonymous class so that method can't be called directly anyway.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195559505
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    Yeah this does seem pretty complicated. I suppose you didn't use 
    
    ```
    public static ArrowBlock serialize(WriteChannel out, ArrowRecordBatch batch)
    ```
    in message serialize to avoid double copy?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195552695
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    +          0 until numPartitions,
    +          handlePartitionBatches)
    --- End diff --
    
    > I did have another idea though, we could stream all partitions to Python out of order, then follow with another small batch of data that contains maps of partitionIndex to orderReceived. Then the partitions could be put into order on the Python side before making the Pandas DataFrame.
    
    This sounds good!


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199371158
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    --- End diff --
    
    Mostly I'm just curious, is there any point in making this a private method?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92432/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    ### Performance Tests - createDataFrame
    
    Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
    measured wall clock time to execute `createDataFrame()` and get the first record. Took the average best time of 5 runs/5 loops each.
    
    Test code
    ```python
    def run():
    	pdf = pd.DataFrame(np.random.rand(10000000, 10))
    	spark.createDataFrame(pdf).first()
    
    for i in range(6):
    	start = time.time()
    	run()
    	elapsed = time.time() - start
    	gc.collect()
    	print("Run %d: %f" % (i, elapsed))
    ```
    
    | Current Master | This PR |
    |---------------------|------------|
    6.234608 | 5.665641
    6.32144 | 5.3475
    6.527859 | 5.370803
    6.95089 | 5.479151
    6.235046 | 5.529167
    
    
    
    | Avg Master | Avg This PR
    -----------|--------------
    6.4539686 | 5.4784524
    
    #### Speedup of 1.178064192
    
    
    
    



---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212171980
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +178,106 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeRecordBatch(
    +      new ReadChannel(Channels.newChannel(in)), allocator)  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    --- End diff --
    
    so this the length of the array of batches, not the number of records in the batch.  The input is split according to the default parallelism config.  So if that is 32, we will have an array of 32 batches and then parallelize those to 32 partitions. `parallelize` might usually have one big array of primitives as the first arg, that you then partition by the number in the second arg, but this is a little different since we are using batches. Does that answer your question?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95145 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95145/testReport)** for PR 21546 at commit [`5549644`](https://github.com/apache/spark/commit/554964465dbcb99cc313620fafb0fc41acfd4304).


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/4041/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195502588
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala ---
    @@ -34,17 +34,36 @@ private[sql] object PythonSQLUtils {
       }
     
       /**
    -   * Python Callable function to convert ArrowPayloads into a [[DataFrame]].
    +   * Python callable function to convert an RDD of serialized ArrowRecordBatches into
    +   * a [[DataFrame]].
        *
    -   * @param payloadRDD A JavaRDD of ArrowPayloads.
    -   * @param schemaString JSON Formatted Schema for ArrowPayloads.
    +   * @param arrowBatchRDD A JavaRDD of serialized ArrowRecordBatches.
    +   * @param schemaString JSON Formatted Spark schema for Arrow batches.
        * @param sqlContext The active [[SQLContext]].
        * @return The converted [[DataFrame]].
        */
    -  def arrowPayloadToDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +  def arrowStreamToDataFrame(
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    ArrowConverters.toDataFrame(payloadRDD, schemaString, sqlContext)
    +    ArrowConverters.toDataFrame(arrowBatchRDD, schemaString, sqlContext)
    +  }
    +
    +  /**
    +   * Python callable function to read a file in Arrow stream format and create a [[DataFrame]]
    +   * using each serialized ArrowRecordBatch as a partition.
    +   *
    +   * @param sqlContext The active [[SQLContext]].
    +   * @param filename File to read the Arrow stream from.
    +   * @param schemaString JSON Formatted Spark schema for Arrow batches.
    +   * @return A new [[DataFrame]].
    +   */
    +  def arrowReadStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String,
    +      schemaString: String): DataFrame = {
    +    JavaSparkContext.fromSparkContext(sqlContext.sparkContext)
    --- End diff --
    
    oops, nothing!  I must have forgot to delete, thanks!


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204319644
  
    --- Diff: python/pyspark/sql/dataframe.py ---
    @@ -2095,9 +2095,11 @@ def toPandas(self):
                             _check_dataframe_localize_timestamps
                         import pyarrow
     
    -                    tables = self._collectAsArrow()
    -                    if tables:
    -                        table = pyarrow.concat_tables(tables)
    +                    # Collect un-ordered list of batches, and list of correct order indices
    +                    batches, batch_order = self._collectAsArrow()
    +                    if batches:
    --- End diff --
    
    Not a big deal at all and personal preference: I would do this like `len(batches) > 0`.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r203790304
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
         """
    -    Serializes bytes as Arrow data with the Arrow file format.
    +    Deserialize a stream of batches followed by batch order information.
         """
     
    -    def dumps(self, batch):
    +    def __init__(self, serializer):
    +        self.serializer = serializer
    +        self.batch_order = None
    +
    +    def dump_stream(self, iterator, stream):
    +        return self.serializer.dump_stream(iterator, stream)
    +
    +    def load_stream(self, stream):
    +        for batch in self.serializer.load_stream(stream):
    +            yield batch
    +        num = read_int(stream)
    +        self.batch_order = []
    +        for i in xrange(num):
    +            index = read_int(stream)
    +            self.batch_order.append(index)
    +        raise StopIteration()
    --- End diff --
    
    @BryanCutler, I think this will be broken in Python 3.7 (see [PEP 479](https://www.python.org/dev/peps/pep-0479)). Shall we just remove this line or explicitly `return`?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95390 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95390/testReport)** for PR 21546 at commit [`ffb47cb`](https://github.com/apache/spark/commit/ffb47cb2d411b91e240ab40cd6bd75b025e417c2).
     * This patch **fails from timeout after a configured wait of \`400m\`**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ArrowEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan)`
      * `case class BatchEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan)`


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95093/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195512218
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala ---
    @@ -1318,18 +1318,52 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll {
         }
       }
     
    -  test("roundtrip payloads") {
    +  test("roundtrip arrow batches") {
         val inputRows = (0 until 9).map { i =>
           InternalRow(i)
         } :+ InternalRow(null)
     
         val schema = StructType(Seq(StructField("int", IntegerType, nullable = true)))
     
         val ctx = TaskContext.empty()
    -    val payloadIter = ArrowConverters.toPayloadIterator(inputRows.toIterator, schema, 0, null, ctx)
    -    val outputRowIter = ArrowConverters.fromPayloadIterator(payloadIter, ctx)
    +    val batchIter = ArrowConverters.toBatchIterator(inputRows.toIterator, schema, 5, null, ctx)
    +    val outputRowIter = ArrowConverters.fromBatchIterator(batchIter, schema, null, ctx)
     
    -    assert(schema == outputRowIter.schema)
    +    var count = 0
    +    outputRowIter.zipWithIndex.foreach { case (row, i) =>
    +      if (i != 9) {
    +        assert(row.getInt(0) == i)
    +      } else {
    +        assert(row.isNullAt(0))
    +      }
    +      count += 1
    +    }
    +
    +    assert(count == inputRows.length)
    +  }
    +
    +  test("ArrowBatchStreamWriter roundtrip") {
    +    val inputRows = (0 until 9).map { i =>
    +      InternalRow(i)
    +    } :+ InternalRow(null)
    +
    +    val schema = StructType(Seq(StructField("int", IntegerType, nullable = true)))
    +
    +    val ctx = TaskContext.empty()
    +    val batchIter = ArrowConverters.toBatchIterator(inputRows.toIterator, schema, 5, null, ctx)
    +
    +    // Write batches to Arrow stream format as a byte array
    +    val out = new ByteArrayOutputStream()
    --- End diff --
    
    done


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92543/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r213835409
  
    --- Diff: python/pyspark/context.py ---
    @@ -494,10 +494,14 @@ def f(split, iterator):
                 c = list(c)    # Make it a list so we can compute its length
             batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024))
             serializer = BatchedSerializer(self._unbatched_serializer, batchSize)
    -        jrdd = self._serialize_to_jvm(c, numSlices, serializer)
    +
    +        def reader_func(temp_filename):
    +            return self._jvm.PythonRDD.readRDDFromFile(self._jsc, temp_filename, numSlices)
    +
    +        jrdd = self._serialize_to_jvm(c, serializer, reader_func)
             return RDD(jrdd, self, serializer)
     
    -    def _serialize_to_jvm(self, data, parallelism, serializer):
    +    def _serialize_to_jvm(self, data, serializer, reader_func):
    --- End diff --
    
    Thanks @BryanCutler , sorry I didn't know where to look for those, they look much better than what I would have added!


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1025/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199484323
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3237,50 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { outputStream =>
    +        val out = new DataOutputStream(outputStream)
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Batches ordered by index of partition + fractional value of batch # in partition
    +        val batchOrder = new ArrayBuffer[Float]()
    +        var partitionCount = 0
    +
    +        // Handler to eagerly write batches to Python out of order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          if (arrowBatches.nonEmpty) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            (0 until arrowBatches.length).foreach { i =>
    --- End diff --
    
    intellij would like you to know about `arrowBatches.indices` :grin:


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r196277485
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    Sure. The main reason that I am not sure about this code is that the code here breaks encapsulation.
    
    If I understand correctly, Arrow reader only supports reading record batch from Channel to Arrow memory, in order to read record batch from Channel to on-heap memory directly, we need to subclass `MessageChannelReader`, overwrite `readNextMessage` to load the the metadata and body of record batch.
    
    Now the main point that I feel not comfortable with this approach:
    The subclass changes the behavior of `readNextMessage` to load  both metadata and body of a record batch, where in the parent class it only loads meta of a record batch. And I think this is the contract of the interface too so this feels a bit hacky.
    
    I am not saying I am totally against this for performance reasons, but considering the code path already involves writing data to disk (so avoid one memory copy won't necessary get us much) and is one of the less frequent operations (pandas DataFrame -> spark DataFrame), I am not sure it's worth it, that's why I suggest resolving this separately so not to block this PR.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Thanks @ueshin and @icexelloss !  I do have some benchmarks that I'll post soon..


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199384074
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3237,50 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { outputStream =>
    +        val out = new DataOutputStream(outputStream)
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Batches ordered by index of partition + fractional value of batch # in partition
    +        val batchOrder = new ArrayBuffer[Float]()
    +        var partitionCount = 0
    +
    +        // Handler to eagerly write batches to Python out of order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          if (arrowBatches.nonEmpty) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            (0 until arrowBatches.length).foreach { i =>
    +              batchOrder.append(index + i / arrowBatches.length)
    --- End diff --
    
    Maybe we should cast to Float when calculating `i / arrowBatches.length`, otherwise the same values are appended?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r203913178
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3349,20 +3385,20 @@ class Dataset[T] private[sql](
         }
       }
     
    -  /** Convert to an RDD of ArrowPayload byte arrays */
    -  private[sql] def toArrowPayload(plan: SparkPlan): RDD[ArrowPayload] = {
    +  /** Convert to an RDD of serialized ArrowRecordBatches. */
    +  private[sql] def getArrowBatchRdd(plan: SparkPlan): RDD[Array[Byte]] = {
    --- End diff --
    
    Yeah, I can't remember why I changed it.. but I think you're right it so I'll change it back.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95137 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95137/testReport)** for PR 21546 at commit [`5549644`](https://github.com/apache/spark/commit/554964465dbcb99cc313620fafb0fc41acfd4304).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199275753
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3237,50 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { outputStream =>
    +        val out = new DataOutputStream(outputStream)
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Batches ordered by index of partition + batch number for that partition
    +        val batchOrder = new ArrayBuffer[Int]()
    +        var partitionCount = 0
    +
    +        // Handler to eagerly write batches to Python out of order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          if (arrowBatches.nonEmpty) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            (0 until arrowBatches.length).foreach { i =>
    +              batchOrder.append(index + i)
    +            }
    +          }
    +          partitionCount += 1
    +
    +          // After last batch, end the stream and write batch order
    +          if (partitionCount == numPartitions) {
    +            batchWriter.end()
    +            out.writeInt(batchOrder.length)
    +            // Batch order indices are from 0 to N-1 batches, sorted by order they arrived
    +            batchOrder.zipWithIndex.sortBy(_._1).foreach { case (_, i) =>
    --- End diff --
    
    Does this logic do what you intend? It interleaves batches.
    
    ```python
    df = spark.range(64).toDF("a")
    df.rdd.getNumPartitions()  # 8
    spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", 4)
    pdf = df.toPandas()
    pdf['a'].values
    # array([ 0,  1,  2,  3,  8,  9, 10, 11,  4,  5,  6,  7, 16, 17, 18, 19, 12,
    #       13, 14, 15, 20, 21, 22, 23, 24, 25, 26, 27, 32, 33, 34, 35, 28, 29,
    #       30, 31, 36, 37, 38, 39, 40, 41, 42, 43, 48, 49, 50, 51, 44, 45, 46,
    #       47, 56, 57, 58, 59, 52, 53, 54, 55, 60, 61, 62, 63])
    ```


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Merged to master.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r202825767
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,59 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
         """
    -    Serializes bytes as Arrow data with the Arrow file format.
    +    Deserialize a stream of batches followed by batch order information.
         """
     
    -    def dumps(self, batch):
    +    def __init__(self, serializer):
    +        self.serializer = serializer
    +        self.batch_order = []
    +
    +    def dump_stream(self, iterator, stream):
    +        return self.serializer.dump_stream(iterator, stream)
    +
    +    def load_stream(self, stream):
    +        for batch in self.serializer.load_stream(stream):
    +            yield batch
    +        num = read_int(stream)
    +        for i in xrange(num):
    +            index = read_int(stream)
    +            self.batch_order.append(index)
    +        raise StopIteration()
    +
    +    def get_batch_order(self):
    --- End diff --
    
    I added an assert and reset `self.batch_order` after calling to ensure `load_stream` is called first and the serializer instance could be used again without retaining state.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3956/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r194904013
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    made https://issues.apache.org/jira/browse/ARROW-2704 to track


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    I removed the WIP because it does not require anything from Arrow 0.10.0, this would only help clean some thing up. @HyukjinKwon, @ueshin what are your thoughts on merging this as is and then doing a followup to clean once Arrow 0.10.0 is released?  There is a chance that we might not get the Arrow upgrade in before Spark 2.4.0, so it would still be nice to get this in.  Thanks!


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r205642926
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
    --- End diff --
    
    Thank you @BryanCutler.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #93465 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93465/testReport)** for PR 21546 at commit [`3224625`](https://github.com/apache/spark/commit/322462586e1c8ac301a44e6da47589a599e423d9).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #93465 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93465/testReport)** for PR 21546 at commit [`3224625`](https://github.com/apache/spark/commit/322462586e1c8ac301a44e6da47589a599e423d9).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204606545
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
    --- End diff --
    
    Thanks for elaborating this, @BryanCutler. Would you mind if I ask to add this separately in a separate PR? I am actually not super sure on this ..


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95083 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95083/testReport)** for PR 21546 at commit [`89d7836`](https://github.com/apache/spark/commit/89d78364d93490b1b301c5ec766e4390bdc0b8a7).


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204324646
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
    --- End diff --
    
    @BryanCutler, just read https://github.com/apache/spark/pull/21546#issuecomment-400824224. How much performance gain was made by this? Looks pretty complicated..


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2423/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195824174
  
    --- Diff: python/pyspark/sql/dataframe.py ---
    @@ -2153,7 +2153,7 @@ def _collectAsArrow(self):
             """
             with SCCallSiteSync(self._sc) as css:
                 sock_info = self._jdf.collectAsArrowToPython()
    -        return list(_load_from_socket(sock_info, ArrowSerializer()))
    +        return list(_load_from_socket(sock_info, ArrowStreamSerializer()))
    --- End diff --
    
    We also need to update the description of `_collectAsArrow()`?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #92395 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92395/testReport)** for PR 21546 at commit [`fe3319b`](https://github.com/apache/spark/commit/fe3319bd7ab290e30f6075a81acd0b17818ad546).


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92395/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r197575182
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    I was just referring to the two static functions in the previous post.  These will contain most of the low level operations inside for stability.  I'm not sure we need a new interface to handle this case, it's probably not a common use case.  I'll just implement what I thought and maybe it will be more clear.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2399/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212178291
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +178,106 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeRecordBatch(
    +      new ReadChannel(Channels.newChannel(in)), allocator)  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    --- End diff --
    
    Ah, sorry. You are right. I misread.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199193918
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val messageLength = readMessageLength(inputChannel)
    +        if (messageLength == 0) {
    +          return null
    +        }
    +
    +        val buffer = ByteBuffer.allocate(messageLength)
    +        val msg = loadMessage(inputChannel, messageLength, buffer)
    --- End diff --
    
    I see.
    
    IMHO the class still touches too many Arrow stream low-level details, but maybe necessary because of the performance improvement.
    
    @BryanCutler does all the speed up of `createDataFrame` comes from avoiding extra copy here?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1242/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r203760078
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3349,20 +3385,20 @@ class Dataset[T] private[sql](
         }
       }
     
    -  /** Convert to an RDD of ArrowPayload byte arrays */
    -  private[sql] def toArrowPayload(plan: SparkPlan): RDD[ArrowPayload] = {
    +  /** Convert to an RDD of serialized ArrowRecordBatches. */
    +  private[sql] def getArrowBatchRdd(plan: SparkPlan): RDD[Array[Byte]] = {
    --- End diff --
    
    @BryanCutler, not a big deal at all but how about `to` like the before? This reminds me of QueryExecution's toRdd and the previous one looks slightly better.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199613477
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    --- End diff --
    
    It is being closed, just by `readArrowStreamFromFile` which owns the stream and consumes the iterator.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199502733
  
    --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala ---
    @@ -398,6 +398,25 @@ private[spark] object PythonRDD extends Logging {
        *         data collected from this job, and the secret for authentication.
        */
       def serveIterator(items: Iterator[_], threadName: String): Array[Any] = {
    +    serveToStream(threadName) { out =>
    +      writeIteratorToStream(items, new DataOutputStream(out))
    +    }
    +  }
    +
    +  /**
    +   * Create a socket server and background thread to execute the block of code
    +   * for the given DataOutputStream.
    +   *
    +   * The socket server can only accept one connection, or close if no connection
    +   * in 15 seconds.
    +   *
    +   * Once a connection comes in, it will execute the block of code and pass in
    +   * the socket output stream.
    +   *
    +   * The thread will terminate after the block of code is executed or any
    +   * exceptions happen.
    +   */
    +  private[spark] def serveToStream(threadName: String)(block: OutputStream => Unit): Array[Any] = {
    --- End diff --
    
    can you change `block` to `writeFunc` or something? `block` makes me think of thread blocking


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r198925610
  
    --- Diff: dev/make-distribution.sh ---
    @@ -168,10 +168,10 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=512m}"
     BUILD_COMMAND=("$MVN" -T 1C clean package -DskipTests $@)
     
     # Actually build the jar
    -echo -e "\nBuilding with..."
    -echo -e "\$ ${BUILD_COMMAND[@]}\n"
    +#echo -e "\nBuilding with..."
    +#echo -e "\$ ${BUILD_COMMAND[@]}\n"
     
    -"${BUILD_COMMAND[@]}"
    +#"${BUILD_COMMAND[@]}"
    --- End diff --
    
    Oops, yeah thanks for pointing it out.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r197149246
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    Btw I don't mind discuss it in this PR too. Also curious what other people think.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    LGTM otherwise.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195497043
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    +          0 until numPartitions,
    +          handlePartitionBatches)
    --- End diff --
    
    > is it better to incrementally run job on partitions in order
    I believe this is how `toLocalIterator` works right?  I tried using that because it does only keep 1 partition in memory at a time, but the performance took quite a hit from the multiple jobs.  I think we should still prioritize performance over memory for `toPandas()` since it's assumed the data to be collect should be relatively small.
    
    I did have another idea though, we could stream all partitions to Python out of order, then follow with another small batch of data that contains maps of partitionIndex to orderReceived.  Then the partitions could be put into order on the Python side before making the Pandas DataFrame.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93455/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #93455 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93455/testReport)** for PR 21546 at commit [`f19f204`](https://github.com/apache/spark/commit/f19f20496a4c5981fb0160878cc22b0d9f76078d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95055 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95055/testReport)** for PR 21546 at commit [`92b8e26`](https://github.com/apache/spark/commit/92b8e2669b96e76c0fa712438f0a7b8f52324dea).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95083 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95083/testReport)** for PR 21546 at commit [`89d7836`](https://github.com/apache/spark/commit/89d78364d93490b1b301c5ec766e4390bdc0b8a7).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class BarrierTaskContext(TaskContext):`
      * `class BarrierTaskInfo(object):`
      * `case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric`
      * `sealed trait StreamingAggregationStateManager extends Serializable `
      * `abstract class StreamingAggregationStateManagerBaseImpl(`
      * `class StreamingAggregationStateManagerImplV1(`
      * `class StreamingAggregationStateManagerImplV2(`


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2650/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199508609
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,59 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
         """
    -    Serializes bytes as Arrow data with the Arrow file format.
    +    Deserialize a stream of batches followed by batch order information.
         """
     
    -    def dumps(self, batch):
    +    def __init__(self, serializer):
    +        self.serializer = serializer
    +        self.batch_order = []
    +
    +    def dump_stream(self, iterator, stream):
    +        return self.serializer.dump_stream(iterator, stream)
    +
    +    def load_stream(self, stream):
    +        for batch in self.serializer.load_stream(stream):
    +            yield batch
    +        num = read_int(stream)
    +        for i in xrange(num):
    +            index = read_int(stream)
    +            self.batch_order.append(index)
    +        raise StopIteration()
    +
    +    def get_batch_order(self):
    --- End diff --
    
    maybe we should initialize `self.batch_order = None`, and add `assert self.batch_order is not None` here.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212161411
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3268,13 +3268,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = toArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    --- End diff --
    
    Is it better `0 to N-1 partitions`?


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r194965715
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala ---
    @@ -1318,18 +1318,52 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll {
         }
       }
     
    -  test("roundtrip payloads") {
    +  test("roundtrip arrow batches") {
         val inputRows = (0 until 9).map { i =>
           InternalRow(i)
         } :+ InternalRow(null)
     
         val schema = StructType(Seq(StructField("int", IntegerType, nullable = true)))
     
         val ctx = TaskContext.empty()
    -    val payloadIter = ArrowConverters.toPayloadIterator(inputRows.toIterator, schema, 0, null, ctx)
    -    val outputRowIter = ArrowConverters.fromPayloadIterator(payloadIter, ctx)
    +    val batchIter = ArrowConverters.toBatchIterator(inputRows.toIterator, schema, 5, null, ctx)
    +    val outputRowIter = ArrowConverters.fromBatchIterator(batchIter, schema, null, ctx)
     
    -    assert(schema == outputRowIter.schema)
    +    var count = 0
    +    outputRowIter.zipWithIndex.foreach { case (row, i) =>
    +      if (i != 9) {
    +        assert(row.getInt(0) == i)
    +      } else {
    +        assert(row.isNullAt(0))
    +      }
    +      count += 1
    +    }
    +
    +    assert(count == inputRows.length)
    +  }
    +
    +  test("ArrowBatchStreamWriter roundtrip") {
    +    val inputRows = (0 until 9).map { i =>
    +      InternalRow(i)
    +    } :+ InternalRow(null)
    +
    +    val schema = StructType(Seq(StructField("int", IntegerType, nullable = true)))
    +
    +    val ctx = TaskContext.empty()
    +    val batchIter = ArrowConverters.toBatchIterator(inputRows.toIterator, schema, 5, null, ctx)
    +
    +    // Write batches to Arrow stream format as a byte array
    +    val out = new ByteArrayOutputStream()
    --- End diff --
    
    Can we use `Utils.tryWithResource { ... }`?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2629/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #91862 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91862/testReport)** for PR 21546 at commit [`4af58f9`](https://github.com/apache/spark/commit/4af58f9539ea12c8c309790001efe497d18f0129).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class ArrowStreamSerializer(Serializer):`


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Thanks @viirya and @HyukjinKwon !


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199497456
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val messageLength = readMessageLength(inputChannel)
    +        if (messageLength == 0) {
    +          return null
    +        }
    +
    +        val buffer = ByteBuffer.allocate(messageLength)
    +        val msg = loadMessage(inputChannel, messageLength, buffer)
    +        val bodyLength = msg.bodyLength().asInstanceOf[Int]
    +
    +        if (msg.headerType() == MessageHeader.RecordBatch) {
    +          val allbuf = ByteBuffer.allocate(4 + messageLength + bodyLength)
    +          allbuf.put(WriteChannel.intToBytes(messageLength))
    +          allbuf.put(buffer)
    +          inputChannel.readFully(allbuf)
    +          allbuf.array()
    +        } else {
    +          if (bodyLength > 0) {
    +            // Skip message body if not a record batch
    --- End diff --
    
    What are the conditions under which we'd expect this to happen?


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r194962360
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala ---
    @@ -34,17 +34,36 @@ private[sql] object PythonSQLUtils {
       }
     
       /**
    -   * Python Callable function to convert ArrowPayloads into a [[DataFrame]].
    +   * Python callable function to convert an RDD of serialized ArrowRecordBatches into
    +   * a [[DataFrame]].
        *
    -   * @param payloadRDD A JavaRDD of ArrowPayloads.
    -   * @param schemaString JSON Formatted Schema for ArrowPayloads.
    +   * @param arrowBatchRDD A JavaRDD of serialized ArrowRecordBatches.
    +   * @param schemaString JSON Formatted Spark schema for Arrow batches.
        * @param sqlContext The active [[SQLContext]].
        * @return The converted [[DataFrame]].
        */
    -  def arrowPayloadToDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +  def arrowStreamToDataFrame(
    --- End diff --
    
    This seems being a private method now?


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r194954051
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    +          0 until numPartitions,
    +          handlePartitionBatches)
    --- End diff --
    
    +1 chunking if we could. I recall Bryan said for grouped UDF we need the entire set.
    
    Also not sure if python side we have any assumption on how much of the partition is in each chunk (there shouldn't be?)



---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95340 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95340/testReport)** for PR 21546 at commit [`2fe46f8`](https://github.com/apache/spark/commit/2fe46f82dc38af972bc0974aca1fd846bcb483e5).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95296 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95296/testReport)** for PR 21546 at commit [`2fe46f8`](https://github.com/apache/spark/commit/2fe46f82dc38af972bc0974aca1fd846bcb483e5).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Thanks @HyukjinKwon and others who reviewed!


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2610/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Eh @BryanCutler, mind resolving conflicts please?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/551/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r196108332
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    @BryanCutler Now I looked at this more I think I understand what are you trying to do: The whole class here is trying to read Arrow record batches from an stream into Java on-heap memory without going through Arrow off-heap memory, is that correct?
    
    Also, this function is only used for pandas DataFrame -> Spark DataFrame?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    LGTM


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    No objection. For me, I need few days to take another look. From a cursory look, seems fine going ahead considering this is an experimental feature and still in progress.
    
    Please go ahead otherwise will take a look and go ahead in few days.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199561983
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3237,50 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { outputStream =>
    +        val out = new DataOutputStream(outputStream)
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Batches ordered by index of partition + fractional value of batch # in partition
    +        val batchOrder = new ArrayBuffer[Float]()
    +        var partitionCount = 0
    +
    +        // Handler to eagerly write batches to Python out of order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          if (arrowBatches.nonEmpty) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            (0 until arrowBatches.length).foreach { i =>
    +              batchOrder.append(index + i / arrowBatches.length)
    --- End diff --
    
    I thought I would need to specify a comparison function, but it looks like Scala can sort a `(Int, Int)` tuple correctly.  I agree this would be best, so I'll change it - thanks!


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r213785551
  
    --- Diff: python/pyspark/context.py ---
    @@ -494,10 +494,14 @@ def f(split, iterator):
                 c = list(c)    # Make it a list so we can compute its length
             batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024))
             serializer = BatchedSerializer(self._unbatched_serializer, batchSize)
    -        jrdd = self._serialize_to_jvm(c, numSlices, serializer)
    +
    +        def reader_func(temp_filename):
    +            return self._jvm.PythonRDD.readRDDFromFile(self._jsc, temp_filename, numSlices)
    +
    +        jrdd = self._serialize_to_jvm(c, serializer, reader_func)
             return RDD(jrdd, self, serializer)
     
    -    def _serialize_to_jvm(self, data, parallelism, serializer):
    +    def _serialize_to_jvm(self, data, serializer, reader_func):
    --- End diff --
    
    hi, sorry for the late review here, and more just a question for myself -- is this aspect tested at all?  IIUC, it would be used in `spark.createDataFrame`, but the tests in session.py don't have arrow enabled, right?
    
    not that I see a bug, mostly just wondering as I was looking at making my own changes here, and it would be nice if I knew there were some tests


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    >Hey @BryanCutler, btw, mind if I ask move the benchmarks into the PR description?
    
    Yeah, I guess it doesn't hurt.  I'll copy them there.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199287248
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3237,50 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { outputStream =>
    +        val out = new DataOutputStream(outputStream)
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Batches ordered by index of partition + batch number for that partition
    +        val batchOrder = new ArrayBuffer[Int]()
    +        var partitionCount = 0
    +
    +        // Handler to eagerly write batches to Python out of order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          if (arrowBatches.nonEmpty) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            (0 until arrowBatches.length).foreach { i =>
    +              batchOrder.append(index + i)
    +            }
    +          }
    +          partitionCount += 1
    +
    +          // After last batch, end the stream and write batch order
    +          if (partitionCount == numPartitions) {
    +            batchWriter.end()
    +            out.writeInt(batchOrder.length)
    +            // Batch order indices are from 0 to N-1 batches, sorted by order they arrived
    +            batchOrder.zipWithIndex.sortBy(_._1).foreach { case (_, i) =>
    --- End diff --
    
    Yeah, looks like something wasn't quite right with the batch indexing... I fixed it and added your test.  Thanks @sethah !


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212171051
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +178,106 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeRecordBatch(
    +      new ReadChannel(Channels.newChannel(in)), allocator)  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    --- End diff --
    
    yup, thanks for catching that


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r194966161
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,24 +184,28 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class ArrowSerializer(Serializer):
    --- End diff --
    
    Should we rename this?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Thanks @HyukjinKwon! Any additional comments @holdenk @sethah @viirya @felixcheung ?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #92432 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92432/testReport)** for PR 21546 at commit [`f6d6620`](https://github.com/apache/spark/commit/f6d662089c223b3382924d53427b08824efdda45).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93137/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #92543 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92543/testReport)** for PR 21546 at commit [`e25acd2`](https://github.com/apache/spark/commit/e25acd2dbd7bdc176d43fa6957cc150edf19bdcd).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95137/
    Test FAILed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95325 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95325/testReport)** for PR 21546 at commit [`2fe46f8`](https://github.com/apache/spark/commit/2fe46f82dc38af972bc0974aca1fd846bcb483e5).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #93137 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93137/testReport)** for PR 21546 at commit [`1308169`](https://github.com/apache/spark/commit/1308169ca7ea44395436e7620a6529a2c5f746cb).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204321600
  
    --- Diff: python/pyspark/sql/dataframe.py ---
    @@ -2146,14 +2148,15 @@ def toPandas(self):
     
         def _collectAsArrow(self):
             """
    -        Returns all records as list of deserialized ArrowPayloads, pyarrow must be installed
    -        and available.
    +        Returns all records as a list of ArrowRecordBatches and batch order as a list of indices,
    +        pyarrow must be installed and available on driver and worker Python environments.
     
             .. note:: Experimental.
             """
    +        ser = BatchOrderSerializer(ArrowStreamSerializer())
             with SCCallSiteSync(self._sc) as css:
                 sock_info = self._jdf.collectAsArrowToPython()
    -        return list(_load_from_socket(sock_info, ArrowSerializer()))
    +        return list(_load_from_socket(sock_info, ser)), ser.get_batch_order_and_reset()
    --- End diff --
    
    Hmmm .. @BryanCutler, would you mind if I ask why this batch order is required?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    @gatorsmile, I'm working on test coverage report and already almost finished. Only left job is to setup a Jenkins job as I talked. cc @shaneknapp 


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195554625
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    +          0 until numPartitions,
    +          handlePartitionBatches)
    --- End diff --
    
    I guess in worst case scenario, the driver still needs to hold all batches in memory. For example, all the batches arrive at the same time.
    
    I wonder if there is a way to:
    (1) Compute all tasks in parallell, once tasks are done, store the result in Block manager on executors.
    (2) Return all block id to the driver
    (3) Driver fetches each block and stream individually.
    
    This way at least the computation is done in parallel, fetching the result sequentially is a trade off of speed vs memory, something we or the user can choose,  but I imagine fetching some 10G - 20G data from executors sequentially shouldn't be too bad.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/150/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195504451
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala ---
    @@ -34,17 +34,36 @@ private[sql] object PythonSQLUtils {
       }
     
       /**
    -   * Python Callable function to convert ArrowPayloads into a [[DataFrame]].
    +   * Python callable function to convert an RDD of serialized ArrowRecordBatches into
    +   * a [[DataFrame]].
        *
    -   * @param payloadRDD A JavaRDD of ArrowPayloads.
    -   * @param schemaString JSON Formatted Schema for ArrowPayloads.
    +   * @param arrowBatchRDD A JavaRDD of serialized ArrowRecordBatches.
    +   * @param schemaString JSON Formatted Spark schema for Arrow batches.
        * @param sqlContext The active [[SQLContext]].
        * @return The converted [[DataFrame]].
        */
    -  def arrowPayloadToDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +  def arrowStreamToDataFrame(
    --- End diff --
    
    oh right, this is only called by the function below so I suppose we don't even need it..


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #91862 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91862/testReport)** for PR 21546 at commit [`4af58f9`](https://github.com/apache/spark/commit/4af58f9539ea12c8c309790001efe497d18f0129).


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    This is a WIP because I had to hack up some of the message processing code in Arrow.  This should be done in Arrow, and then it can be cleaned up here.  I will make these changes for version 0.10.0 and complete this once we have upgraded.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199613741
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3237,50 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { outputStream =>
    +        val out = new DataOutputStream(outputStream)
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Batches ordered by index of partition + fractional value of batch # in partition
    +        val batchOrder = new ArrayBuffer[Float]()
    +        var partitionCount = 0
    +
    +        // Handler to eagerly write batches to Python out of order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          if (arrowBatches.nonEmpty) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            (0 until arrowBatches.length).foreach { i =>
    --- End diff --
    
    Great, Intellij is trying to steal my job! :fearful: 


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2655/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r213818193
  
    --- Diff: python/pyspark/context.py ---
    @@ -494,10 +494,14 @@ def f(split, iterator):
                 c = list(c)    # Make it a list so we can compute its length
             batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024))
             serializer = BatchedSerializer(self._unbatched_serializer, batchSize)
    -        jrdd = self._serialize_to_jvm(c, numSlices, serializer)
    +
    +        def reader_func(temp_filename):
    +            return self._jvm.PythonRDD.readRDDFromFile(self._jsc, temp_filename, numSlices)
    +
    +        jrdd = self._serialize_to_jvm(c, serializer, reader_func)
             return RDD(jrdd, self, serializer)
     
    -    def _serialize_to_jvm(self, data, parallelism, serializer):
    +    def _serialize_to_jvm(self, data, serializer, reader_func):
    --- End diff --
    
    Hey @squito , yes that's correct this is in the path that `ArrowTests` with `createDataFrame` tests. These tests are skipped if pyarrow is not installed, but for our Jenkins tests it is installed under the Python 3.5 env so it gets tested there.  
    
    It's a little subtle to see that they were run since the test output shows only when tests are skipped. You can see that for Python 2.7 `ArrowTests` show as skipped, but for 3.5 it does not.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Yes, that is the worst case. If there is some bug with types/schema then
    there is an automatic fallback to the non-arrow code path too
    
    On Tue, Aug 28, 2018, 7:16 PM Xiao Li <no...@github.com> wrote:
    
    > @BryanCutler <https://github.com/BryanCutler> The worst case is to turn
    > off spark.sql.execution.arrow.enabled, if the new code path has a bug,
    > right?
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub
    > <https://github.com/apache/spark/pull/21546#issuecomment-416800476>, or mute
    > the thread
    > <https://github.com/notifications/unsubscribe-auth/AEUwdeFLnFzURZmVk4wwwcJJhBumZJeqks5uVfligaJpZM4UlMKq>
    > .
    >



---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    @HyukjinKwon I redid the benchmarks for `toPandas` with the current code and updated the description. It's not a huge speedup now, but definitely does improve some. I'll also followup with another PR with the out-of-order batches to improve this even further. Let me know if this looks ok to you (pending tests). Thanks!


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2588/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2472/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95363 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95363/testReport)** for PR 21546 at commit [`2fe46f8`](https://github.com/apache/spark/commit/2fe46f82dc38af972bc0974aca1fd846bcb483e5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95310 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95310/testReport)** for PR 21546 at commit [`2fe46f8`](https://github.com/apache/spark/commit/2fe46f82dc38af972bc0974aca1fd846bcb483e5).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95310/
    Test FAILed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

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


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95397 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95397/testReport)** for PR 21546 at commit [`ffb47cb`](https://github.com/apache/spark/commit/ffb47cb2d411b91e240ab40cd6bd75b025e417c2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class ArrowEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan)`
      * `case class BatchEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan)`


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    It's not a bug. Here's what's going on now:
    
    He proposed a way to write out regardless of partition order, I suggested to revert that part back (see https://github.com/apache/spark/pull/21546#discussion_r204324646) since the code path looks being complicated and looks that can orthogonally proceed, which can be also applied to other PySpark code path.
    
    This one still has some improvements (see https://github.com/apache/spark/pull/21546#issuecomment-416373315).



---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212170606
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +178,106 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeRecordBatch(
    +      new ReadChannel(Channels.newChannel(in)), allocator)  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val msgMetadata = MessageSerializer.readMessage(new ReadChannel(in))
    +        if (msgMetadata == null) {
    +          return null
    +        }
    +
    +        // Get the length of the body, which has not be read at this point
    +        val bodyLength = msgMetadata.getMessageBodyLength.toInt
    +
    +        // Only care about RecordBatch data, skip Schema and unsupported Dictionary messages
    +        if (msgMetadata.getMessage.headerType() == MessageHeader.RecordBatch) {
    +
    +          // Create output backed by buffer to hold msg length (int32), msg metadata, msg body
    +          val bbout = new ByteBufferOutputStream(4 + msgMetadata.getMessageLength + bodyLength)
    --- End diff --
    
    I'll add some more details about what this is doing


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/67/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    retest this please


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #92233 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92233/testReport)** for PR 21546 at commit [`b971e42`](https://github.com/apache/spark/commit/b971e42ee2973ba72a7668a39e1bf4c3de919289).


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r197552655
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    Yeah.. one way to do it is to write a new MessageReader interface to read Arrow message from a Channel:
    
    ```
    public class OnHeapMessageChannelReader {
      /**
       * Read the next message in the sequence.
       *
       * @return The read message or null if reached the end of the message sequence
       * @throws IOException
       */
      Message readNextMessage() throws IOException;
    
      /**
       * When a message is followed by a body of data, read that data into an ArrowBuf. This should
       * only be called when a Message has a body length > 0.
       *
       * @param message Read message that is followed by a body of data
       * @param allocator BufferAllocator to allocate memory for body data
       * @return An ArrowBuf containing the body of the message that was read
       * @throws IOException
       */
       ByteBuffer readMessageBody(Message message) throws IOException;
    
       ...
    }
    ```
    
    We might need to duplicate some logic in https://github.com/apache/arrow/blob/master/java/vector/src/main/java/org/apache/arrow/vector/ipc/message/MessageChannelReader.java#L33 
    
    For record batches, it's not too bad because the logic is pretty simple, but the down side is we will be using low level APIs of Arrow, which might not be guaranteed to be stable .
    
    @BryanCutler what kind of static function do you think we need to add on the Arrow side?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    retest this please


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r196247663
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: simplify in super class
    +    class RecordBatchMessageReader(inputChannel: SeekableByteChannel) {
    --- End diff --
    
    I think we should definitely avoid extra copies of the data in the JVM if possible, since we are trying to be efficient here.  This process doesn't really seem complex to me, the specialized code here is about 10 lines and just reads bytes from an input stream to a byte buffer.  Can you clarify a bit more on why you think this should be in a separate PR? 


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95093 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95093/testReport)** for PR 21546 at commit [`89d7836`](https://github.com/apache/spark/commit/89d78364d93490b1b301c5ec766e4390bdc0b8a7).


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199478745
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala ---
    @@ -34,17 +33,19 @@ private[sql] object PythonSQLUtils {
       }
     
       /**
    -   * Python Callable function to convert ArrowPayloads into a [[DataFrame]].
    +   * Python callable function to read a file in Arrow stream format and create a [[DataFrame]]
    +   * using each serialized ArrowRecordBatch as a partition.
        *
    -   * @param payloadRDD A JavaRDD of ArrowPayloads.
    -   * @param schemaString JSON Formatted Schema for ArrowPayloads.
        * @param sqlContext The active [[SQLContext]].
    -   * @return The converted [[DataFrame]].
    +   * @param filename File to read the Arrow stream from.
    +   * @param schemaString JSON Formatted Spark schema for Arrow batches.
    +   * @return A new [[DataFrame]].
        */
    -  def arrowPayloadToDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    -      schemaString: String,
    -      sqlContext: SQLContext): DataFrame = {
    -    ArrowConverters.toDataFrame(payloadRDD, schemaString, sqlContext)
    +  def arrowReadStreamFromFile(
    --- End diff --
    
    Can we call it `arrowFileToDataFrame` or something... `arrowReadStreamFromFile` and `readArrowStreamFromFile` are just too similar...


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r213819766
  
    --- Diff: python/pyspark/context.py ---
    @@ -494,10 +494,14 @@ def f(split, iterator):
                 c = list(c)    # Make it a list so we can compute its length
             batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024))
             serializer = BatchedSerializer(self._unbatched_serializer, batchSize)
    -        jrdd = self._serialize_to_jvm(c, numSlices, serializer)
    +
    +        def reader_func(temp_filename):
    +            return self._jvm.PythonRDD.readRDDFromFile(self._jsc, temp_filename, numSlices)
    +
    +        jrdd = self._serialize_to_jvm(c, serializer, reader_func)
             return RDD(jrdd, self, serializer)
     
    -    def _serialize_to_jvm(self, data, parallelism, serializer):
    +    def _serialize_to_jvm(self, data, serializer, reader_func):
    --- End diff --
    
    I made https://issues.apache.org/jira/browse/SPARK-25272 which will give a more clear output that the ArrowTests were run.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r198660818
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val messageLength = readMessageLength(inputChannel)
    +        if (messageLength == 0) {
    +          return null
    +        }
    +
    +        val buffer = ByteBuffer.allocate(messageLength)
    +        val msg = loadMessage(inputChannel, messageLength, buffer)
    --- End diff --
    
    The only issue with this is that it is aware that a message is preceded by the message length and that a length of zero indicates no more messages.  Ideally, this logic would be abstracted to Arrow...


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #91733 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91733/testReport)** for PR 21546 at commit [`a5a1fbe`](https://github.com/apache/spark/commit/a5a1fbe7121c5b0dd93876a56c29ad17dcd9b168).


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195499089
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala ---
    @@ -34,17 +34,36 @@ private[sql] object PythonSQLUtils {
       }
     
       /**
    -   * Python Callable function to convert ArrowPayloads into a [[DataFrame]].
    +   * Python callable function to convert an RDD of serialized ArrowRecordBatches into
    +   * a [[DataFrame]].
        *
    -   * @param payloadRDD A JavaRDD of ArrowPayloads.
    -   * @param schemaString JSON Formatted Schema for ArrowPayloads.
    +   * @param arrowBatchRDD A JavaRDD of serialized ArrowRecordBatches.
    +   * @param schemaString JSON Formatted Spark schema for Arrow batches.
        * @param sqlContext The active [[SQLContext]].
        * @return The converted [[DataFrame]].
        */
    -  def arrowPayloadToDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +  def arrowStreamToDataFrame(
    --- End diff --
    
    it's public so it can be called in Python with Py4j


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/423/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/593/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    > Btw, do you have benchmark number or something yet?
    
    ++1


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #93455 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93455/testReport)** for PR 21546 at commit [`f19f204`](https://github.com/apache/spark/commit/f19f20496a4c5981fb0160878cc22b0d9f76078d).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93465/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2599/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    For now, you can ruin https://github.com/apache/spark/blob/master/python/run-tests-with-coverage script to check the coverage.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212162307
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +178,106 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeRecordBatch(
    +      new ReadChannel(Channels.newChannel(in)), allocator)  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    --- End diff --
    
    nit: i would do `tryWithResource`


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r213825858
  
    --- Diff: python/pyspark/context.py ---
    @@ -494,10 +494,14 @@ def f(split, iterator):
                 c = list(c)    # Make it a list so we can compute its length
             batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024))
             serializer = BatchedSerializer(self._unbatched_serializer, batchSize)
    -        jrdd = self._serialize_to_jvm(c, numSlices, serializer)
    +
    +        def reader_func(temp_filename):
    +            return self._jvm.PythonRDD.readRDDFromFile(self._jsc, temp_filename, numSlices)
    +
    +        jrdd = self._serialize_to_jvm(c, serializer, reader_func)
             return RDD(jrdd, self, serializer)
     
    -    def _serialize_to_jvm(self, data, parallelism, serializer):
    +    def _serialize_to_jvm(self, data, serializer, reader_func):
    --- End diff --
    
    To be honest, I worry about the test coverage of PySpark in general. Anybody in PySpark can lead the effort to propose a solution for improving the test coverage? 


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95055/
    Test FAILed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204484728
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
    --- End diff --
    
    Yeah, the performance gain by sending out of order batches was small, but the reason this was done was to improve memory usage in the driver JVM.  Before this it still had a worst case of buffering the entire dataset in the JVM, but now nothing is buffered and partitions are immediately sent to Python.  I think that's a huge improvement that is worth the additional complexity. This method might even be applicable to a `collect()` in Python also.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199482134
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    --- End diff --
    
    do we not need to close this when the iterator has been consumed?


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199482021
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    --- End diff --
    
    delete extra line


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2480/
    Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r196172708
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    --- End diff --
    
    I tried playing around with that a while ago and can't remember if there was some problem, but I'll give it another shot.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    cc @ueshin 



---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199496002
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3237,50 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { outputStream =>
    +        val out = new DataOutputStream(outputStream)
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Batches ordered by index of partition + fractional value of batch # in partition
    +        val batchOrder = new ArrayBuffer[Float]()
    +        var partitionCount = 0
    +
    +        // Handler to eagerly write batches to Python out of order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          if (arrowBatches.nonEmpty) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            (0 until arrowBatches.length).foreach { i =>
    +              batchOrder.append(index + i / arrowBatches.length)
    --- End diff --
    
    This code: `(0 until array.length).map(i => i / array.length)` is guaranteed to produce only zero values isn't it? The code works, since `sortBy` evidently preserves the ordering of equal elements, but you may as well do `batchOrder.append(index)` since it's the same. 


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    ### Memory Improvements
    
    **toPandas()**
    
    The most significant improvement is reduction of the upper bound space complexity in the JVM driver.  Before, the entire dataset was collected in the JVM first before sending it to Python.  With this change, as soon as a partition is collected, the result handler immediately sends it to Python, so the upper bound is the size of the largest partition.  Also, using the Arrow stream format is more efficient because the schema is written once per stream, followed by record batches.  The schema is now only send from driver JVM to Python.  Before, multiple Arrow file formats were used that each contained the schema.  This duplicated schema was created in the executors, sent to the driver JVM, and then Python where all but the first one received are discarded.
    
    I verified the upper bound limit by running a test that would collect data that would exceed the amount of driver JVM memory available.  Using these settings on a standalone cluster:
    ```
    spark.driver.memory              1g
    spark.executor.memory            5g
    spark.sql.execution.arrow.enabled  true
    spark.sql.execution.arrow.fallback.enabled  false
    spark.sql.execution.arrow.maxRecordsPerBatch 0
    spark.driver.maxResultSize 2g
    ```
    
    Test code:
    ```python
    from pyspark.sql.functions import rand
    df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand())
    df.toPandas()
    ```
    
    This makes total data size of 33554432×8×4 = 1073741824
    
    With the current master, it fails with OOM but passes using this PR.
    
    
    **createDataFrame()**
    
    No significant change in memory except that using the stream format instead of separate file formats avoids duplicated the schema, similar to toPandas above.  The process of reading the stream and parallelizing the batches does cause the record batch message metadata to be copied, but it's size is insignificant.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195501843
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala ---
    @@ -1318,18 +1318,52 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll {
         }
       }
     
    -  test("roundtrip payloads") {
    +  test("roundtrip arrow batches") {
         val inputRows = (0 until 9).map { i =>
           InternalRow(i)
         } :+ InternalRow(null)
     
         val schema = StructType(Seq(StructField("int", IntegerType, nullable = true)))
     
         val ctx = TaskContext.empty()
    -    val payloadIter = ArrowConverters.toPayloadIterator(inputRows.toIterator, schema, 0, null, ctx)
    -    val outputRowIter = ArrowConverters.fromPayloadIterator(payloadIter, ctx)
    +    val batchIter = ArrowConverters.toBatchIterator(inputRows.toIterator, schema, 5, null, ctx)
    +    val outputRowIter = ArrowConverters.fromBatchIterator(batchIter, schema, null, ctx)
     
    -    assert(schema == outputRowIter.schema)
    +    var count = 0
    +    outputRowIter.zipWithIndex.foreach { case (row, i) =>
    +      if (i != 9) {
    +        assert(row.getInt(0) == i)
    +      } else {
    +        assert(row.isNullAt(0))
    +      }
    +      count += 1
    +    }
    +
    +    assert(count == inputRows.length)
    +  }
    +
    +  test("ArrowBatchStreamWriter roundtrip") {
    +    val inputRows = (0 until 9).map { i =>
    +      InternalRow(i)
    +    } :+ InternalRow(null)
    +
    +    val schema = StructType(Seq(StructField("int", IntegerType, nullable = true)))
    +
    +    val ctx = TaskContext.empty()
    +    val batchIter = ArrowConverters.toBatchIterator(inputRows.toIterator, schema, 5, null, ctx)
    +
    +    // Write batches to Arrow stream format as a byte array
    +    val out = new ByteArrayOutputStream()
    --- End diff --
    
    This doesn't actually need to be closed, but I should be closing the DataOutputStream, so I'll put that in tryWithResource


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204960838
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
    --- End diff --
    
    Ah, okay. I think I understood the benefit. But my impression is that this is something we already were doing. Also, if this is something we could apply to other functionalities too, then it sounded to me a bit of orthogonal work to do separately.
    
    Another concern is, for example, how much we'd likely hit this OOM because I usually expect the data for createDataFrame from Pandas DataFrame or toPandas is likely be small.
    
    If the changes were small, then it would have been okay to me but kind of large changes and looks affecting many codes from Scala side to Python side.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204635055
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
         """
    -    Serializes bytes as Arrow data with the Arrow file format.
    +    Deserialize a stream of batches followed by batch order information.
         """
     
    -    def dumps(self, batch):
    +    def __init__(self, serializer):
    +        self.serializer = serializer
    +        self.batch_order = None
    +
    +    def dump_stream(self, iterator, stream):
    +        return self.serializer.dump_stream(iterator, stream)
    +
    +    def load_stream(self, stream):
    +        for batch in self.serializer.load_stream(stream):
    +            yield batch
    +        num = read_int(stream)
    +        self.batch_order = []
    +        for i in xrange(num):
    +            index = read_int(stream)
    +            self.batch_order.append(index)
    +        raise StopIteration()
    --- End diff --
    
    this seems important...


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    so... can we still target this for 2.4? sounds like very good to have and complimentary to hydrogen 


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r198723785
  
    --- Diff: dev/make-distribution.sh ---
    @@ -168,10 +168,10 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=512m}"
     BUILD_COMMAND=("$MVN" -T 1C clean package -DskipTests $@)
     
     # Actually build the jar
    -echo -e "\nBuilding with..."
    -echo -e "\$ ${BUILD_COMMAND[@]}\n"
    +#echo -e "\nBuilding with..."
    +#echo -e "\$ ${BUILD_COMMAND[@]}\n"
     
    -"${BUILD_COMMAND[@]}"
    +#"${BUILD_COMMAND[@]}"
    --- End diff --
    
    did you comment this out for local test?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    SGTM except the discussion going on in https://github.com/apache/spark/pull/21546#discussion_r204324646


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r212170997
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3268,13 +3268,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = toArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    --- End diff --
    
    It's not necessary to buffer the first partition because it can be sent to Python right away, so only need an array of size N-1


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95137 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95137/testReport)** for PR 21546 at commit [`5549644`](https://github.com/apache/spark/commit/554964465dbcb99cc313620fafb0fc41acfd4304).


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204868891
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
    --- End diff --
    
    Yeah, I could separate this but is there anything I can do to alleviate your concern?  I'm not sure I'll have the time to try to make another PR before 2.4.0 code freeze and I think this is a really useful memory optimization to help prevent OOM in the driver JVM.  Also, I might have to rerun the benchmarks here, just to be thorough, because the previous ones were from quite a while ago.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195852498
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    --- End diff --
    
    Can we call `handlePartitionBatches` here before `it.toArray`? I'd do `it.toArray` as lazy as possible.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r194963031
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,131 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and return an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(sqlContext: SQLContext, filename: String):
    +  JavaRDD[Array[Byte]] = {
    --- End diff --
    
    indentation


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95145/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    >@BryanCutler, this takes longer then I thought. Will complete my review till this weekend. For clarification, still no objection about merging it in orthogonally with my review.
    
    No problem, thanks @HyukjinKwon !


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1248/
    Test PASSed.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Yea.. I hope so. only one comment left.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r213826584
  
    --- Diff: python/pyspark/context.py ---
    @@ -494,10 +494,14 @@ def f(split, iterator):
                 c = list(c)    # Make it a list so we can compute its length
             batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024))
             serializer = BatchedSerializer(self._unbatched_serializer, batchSize)
    -        jrdd = self._serialize_to_jvm(c, numSlices, serializer)
    +
    +        def reader_func(temp_filename):
    +            return self._jvm.PythonRDD.readRDDFromFile(self._jsc, temp_filename, numSlices)
    +
    +        jrdd = self._serialize_to_jvm(c, serializer, reader_func)
             return RDD(jrdd, self, serializer)
     
    -    def _serialize_to_jvm(self, data, parallelism, serializer):
    +    def _serialize_to_jvm(self, data, serializer, reader_func):
    --- End diff --
    
    Although most parts in PySpark should be guaranteed by Spark Core and SQL, PySpark starts to have more and more PySpark-only stuffs. I am not very sure how well they are tested. 


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95083/
    Test FAILed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r211964996
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +178,106 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeRecordBatch(
    +      new ReadChannel(Channels.newChannel(in)), allocator)  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val msgMetadata = MessageSerializer.readMessage(new ReadChannel(in))
    +        if (msgMetadata == null) {
    +          return null
    +        }
    +
    +        // Get the length of the body, which has not be read at this point
    +        val bodyLength = msgMetadata.getMessageBodyLength.toInt
    +
    +        // Only care about RecordBatch data, skip Schema and unsupported Dictionary messages
    +        if (msgMetadata.getMessage.headerType() == MessageHeader.RecordBatch) {
    +
    +          // Create output backed by buffer to hold msg length (int32), msg metadata, msg body
    +          val bbout = new ByteBufferOutputStream(4 + msgMetadata.getMessageLength + bodyLength)
    --- End diff --
    
    Add a comment that this is the deserialized form of an Arrow Record Batch? 


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    @HyukjinKwon @felixcheung , I reverted the out-of-order batching for now (we can look at that in another PR) and cleaned up things after the Arrow 0.10.0 upgrade.  If this looks good to you guys, I'd like to rerun some of the benchmarks double-check before merging. Thanks!


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #92395 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92395/testReport)** for PR 21546 at commit [`fe3319b`](https://github.com/apache/spark/commit/fe3319bd7ab290e30f6075a81acd0b17818ad546).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class BatchOrderSerializer(Serializer):`


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199615244
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -38,70 +39,75 @@ import org.apache.spark.util.Utils
     
     
     /**
    - * Store Arrow data in a form that can be serialized by Spark and served to a Python process.
    + * Writes serialized ArrowRecordBatches to a DataOutputStream in the Arrow stream format.
      */
    -private[sql] class ArrowPayload private[sql] (payload: Array[Byte]) extends Serializable {
    +private[sql] class ArrowBatchStreamWriter(
    +    schema: StructType,
    +    out: OutputStream,
    +    timeZoneId: String) {
     
    -  /**
    -   * Convert the ArrowPayload to an ArrowRecordBatch.
    -   */
    -  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
    -    ArrowConverters.byteArrayToBatch(payload, allocator)
    -  }
    +  val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId)
    +  val writeChannel = new WriteChannel(Channels.newChannel(out))
    +
    +  // Write the Arrow schema first, before batches
    +  MessageSerializer.serialize(writeChannel, arrowSchema)
     
       /**
    -   * Get the ArrowPayload as a type that can be served to Python.
    +   * Consume iterator to write each serialized ArrowRecordBatch to the stream.
        */
    -  def asPythonSerializable: Array[Byte] = payload
    -}
    -
    -/**
    - * Iterator interface to iterate over Arrow record batches and return rows
    - */
    -private[sql] trait ArrowRowIterator extends Iterator[InternalRow] {
    +  def writeBatches(arrowBatchIter: Iterator[Array[Byte]]): Unit = {
    +    arrowBatchIter.foreach { batchBytes =>
    +      writeChannel.write(batchBytes)
    +    }
    +  }
     
       /**
    -   * Return the schema loaded from the Arrow record batch being iterated over
    +   * End the Arrow stream, does not close output stream.
        */
    -  def schema: StructType
    +  def end(): Unit = {
    +    // Write End of Stream
    --- End diff --
    
    Since right now it's just writing a `0`, I think it's useful to comment that this means the EOS code.  I have a TODO here to wrap this call in an Arrow function that will be more clear, then we wouldn't need a comment.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #91733 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91733/testReport)** for PR 21546 at commit [`a5a1fbe`](https://github.com/apache/spark/commit/a5a1fbe7121c5b0dd93876a56c29ad17dcd9b168).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195498764
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    +          0 until numPartitions,
    +          handlePartitionBatches)
    --- End diff --
    
    > +1 chunking if we could. I recall Bryan said for grouped UDF we need the entire set.
    
    This still keeps Arrow record batches chunked within each partition, which can help the executor memory, but doesn't do anything for the driver side because we still need to collect the entire partition in the driver JVM.
    
    > Also not sure if python side we have any assumption on how much of the partition is in each chunk (there shouldn't be?)
    No, Python doesn't care how many chunks the data is in, it's handled by pyarrow


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195501939
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,24 +184,28 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class ArrowSerializer(Serializer):
    --- End diff --
    
    Maybe `ArrowStreamSerializer`?


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r213815841
  
    --- Diff: python/pyspark/context.py ---
    @@ -494,10 +494,14 @@ def f(split, iterator):
                 c = list(c)    # Make it a list so we can compute its length
             batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024))
             serializer = BatchedSerializer(self._unbatched_serializer, batchSize)
    -        jrdd = self._serialize_to_jvm(c, numSlices, serializer)
    +
    +        def reader_func(temp_filename):
    +            return self._jvm.PythonRDD.readRDDFromFile(self._jsc, temp_filename, numSlices)
    +
    +        jrdd = self._serialize_to_jvm(c, serializer, reader_func)
             return RDD(jrdd, self, serializer)
     
    -    def _serialize_to_jvm(self, data, parallelism, serializer):
    +    def _serialize_to_jvm(self, data, serializer, reader_func):
    --- End diff --
    
    (if not, I can try to address this in some other work)


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by BryanCutler <gi...@git.apache.org>.
Github user BryanCutler commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    @gatorsmile , this is just the format for Arrow IPC between the JVM and Python process and although it used the Arrow File format, there is nothing persisted. There is no real reason to keep both formats, the stream format is better for our purposes and it's already what is being used for `pandas_udf`s, so there is unlikely a bug in the Arrow format itself. As with any change, a bug is possible but this has been tested pretty thouroughly and trying to keep the old code would get really messy and complicated.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199618628
  
    --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala ---
    @@ -398,6 +398,25 @@ private[spark] object PythonRDD extends Logging {
        *         data collected from this job, and the secret for authentication.
        */
       def serveIterator(items: Iterator[_], threadName: String): Array[Any] = {
    +    serveToStream(threadName) { out =>
    +      writeIteratorToStream(items, new DataOutputStream(out))
    +    }
    +  }
    +
    +  /**
    +   * Create a socket server and background thread to execute the block of code
    +   * for the given DataOutputStream.
    +   *
    +   * The socket server can only accept one connection, or close if no connection
    +   * in 15 seconds.
    +   *
    +   * Once a connection comes in, it will execute the block of code and pass in
    +   * the socket output stream.
    +   *
    +   * The thread will terminate after the block of code is executed or any
    +   * exceptions happen.
    +   */
    +  private[spark] def serveToStream(threadName: String)(block: OutputStream => Unit): Array[Any] = {
    --- End diff --
    
    Yeah, I think I started off with `writeFunc`.. I agree sounds a bit better


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199498622
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -38,70 +39,75 @@ import org.apache.spark.util.Utils
     
     
     /**
    - * Store Arrow data in a form that can be serialized by Spark and served to a Python process.
    + * Writes serialized ArrowRecordBatches to a DataOutputStream in the Arrow stream format.
      */
    -private[sql] class ArrowPayload private[sql] (payload: Array[Byte]) extends Serializable {
    +private[sql] class ArrowBatchStreamWriter(
    +    schema: StructType,
    +    out: OutputStream,
    +    timeZoneId: String) {
     
    -  /**
    -   * Convert the ArrowPayload to an ArrowRecordBatch.
    -   */
    -  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
    -    ArrowConverters.byteArrayToBatch(payload, allocator)
    -  }
    +  val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId)
    +  val writeChannel = new WriteChannel(Channels.newChannel(out))
    +
    +  // Write the Arrow schema first, before batches
    +  MessageSerializer.serialize(writeChannel, arrowSchema)
     
       /**
    -   * Get the ArrowPayload as a type that can be served to Python.
    +   * Consume iterator to write each serialized ArrowRecordBatch to the stream.
        */
    -  def asPythonSerializable: Array[Byte] = payload
    -}
    -
    -/**
    - * Iterator interface to iterate over Arrow record batches and return rows
    - */
    -private[sql] trait ArrowRowIterator extends Iterator[InternalRow] {
    +  def writeBatches(arrowBatchIter: Iterator[Array[Byte]]): Unit = {
    +    arrowBatchIter.foreach { batchBytes =>
    +      writeChannel.write(batchBytes)
    +    }
    +  }
     
       /**
    -   * Return the schema loaded from the Arrow record batch being iterated over
    +   * End the Arrow stream, does not close output stream.
        */
    -  def schema: StructType
    +  def end(): Unit = {
    +    // Write End of Stream
    --- End diff --
    
    this comment can be removed I think


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    @BryanCutler The worst case is to turn off `spark.sql.execution.arrow.enabled`, if the new code path has a bug, right?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #92543 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92543/testReport)** for PR 21546 at commit [`e25acd2`](https://github.com/apache/spark/commit/e25acd2dbd7bdc176d43fa6957cc150edf19bdcd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #92487 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92487/testReport)** for PR 21546 at commit [`8b814b7`](https://github.com/apache/spark/commit/8b814b76d48e9727fafcd85723949142a3658617).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r196172265
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,31 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class ArrowStreamSerializer(Serializer):
    --- End diff --
    
    That was my thought too.  It's pretty close, although we do some different handling  `ArrowStreamPandasSerializer` that needs to fit in somewhere.  Maybe we can look into this as a followup?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #92432 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92432/testReport)** for PR 21546 at commit [`f6d6620`](https://github.com/apache/spark/commit/f6d662089c223b3382924d53427b08824efdda45).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    retest this please


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

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

    https://github.com/apache/spark/pull/21546
  
    Build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199241520
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val messageLength = readMessageLength(inputChannel)
    +        if (messageLength == 0) {
    +          return null
    +        }
    +
    +        val buffer = ByteBuffer.allocate(messageLength)
    +        val msg = loadMessage(inputChannel, messageLength, buffer)
    --- End diff --
    
    I'll propose something different under https://github.com/apache/arrow/pull/2139, we can continue to discuss there


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r195822151
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,31 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class ArrowStreamSerializer(Serializer):
    --- End diff --
    
    I'm wondering if we can reuse this for `ArrowStreamPandasSerializer`?


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95093 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95093/testReport)** for PR 21546 at commit [`89d7836`](https://github.com/apache/spark/commit/89d78364d93490b1b301c5ec766e4390bdc0b8a7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class BarrierTaskContext(TaskContext):`
      * `class BarrierTaskInfo(object):`
      * `case class StateStoreCustomSumMetric(name: String, desc: String) extends StateStoreCustomMetric`
      * `sealed trait StreamingAggregationStateManager extends Serializable `
      * `abstract class StreamingAggregationStateManagerBaseImpl(`
      * `class StreamingAggregationStateManagerImplV1(`
      * `class StreamingAggregationStateManagerImplV2(`


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95296 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95296/testReport)** for PR 21546 at commit [`2fe46f8`](https://github.com/apache/spark/commit/2fe46f82dc38af972bc0974aca1fd846bcb483e5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

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

    https://github.com/apache/spark/pull/21546
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r205624568
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
    --- End diff --
    
    @HyukjinKwon those are fair questions, not sure I'm going to have the time to do anything about it right now, but I'll circle back to this later.


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95055 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95055/testReport)** for PR 21546 at commit [`92b8e26`](https://github.com/apache/spark/commit/92b8e2669b96e76c0fa712438f0a7b8f52324dea).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #95363 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95363/testReport)** for PR 21546 at commit [`2fe46f8`](https://github.com/apache/spark/commit/2fe46f82dc38af972bc0974aca1fd846bcb483e5).


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r199001584
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val messageLength = readMessageLength(inputChannel)
    +        if (messageLength == 0) {
    +          return null
    +        }
    +
    +        val buffer = ByteBuffer.allocate(messageLength)
    +        val msg = loadMessage(inputChannel, messageLength, buffer)
    --- End diff --
    
    We can't really use it as is for a couple of reasons.  It just returns a flatbuffer `Message`, which means that to get it back into another buffer, it requires doing a lot of work with a `FlatBufferBuilder` and some special low level logic that we don't want here.  Arrow has a method `serialize(WriteChannel out, Schema schema)` but then we would have to deserialize only to serialize again.  So it's best to just use the original buffer containing the message.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r198994936
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala ---
    @@ -183,34 +182,111 @@ private[sql] object ArrowConverters {
       }
     
       /**
    -   * Convert a byte array to an ArrowRecordBatch.
    +   * Load a serialized ArrowRecordBatch.
        */
    -  private[arrow] def byteArrayToBatch(
    +  private[arrow] def loadBatch(
           batchBytes: Array[Byte],
           allocator: BufferAllocator): ArrowRecordBatch = {
    -    val in = new ByteArrayReadableSeekableByteChannel(batchBytes)
    -    val reader = new ArrowFileReader(in, allocator)
    -
    -    // Read a batch from a byte stream, ensure the reader is closed
    -    Utils.tryWithSafeFinally {
    -      val root = reader.getVectorSchemaRoot  // throws IOException
    -      val unloader = new VectorUnloader(root)
    -      reader.loadNextBatch()  // throws IOException
    -      unloader.getRecordBatch
    -    } {
    -      reader.close()
    -    }
    +    val in = new ByteArrayInputStream(batchBytes)
    +    MessageSerializer.deserializeMessageBatch(new ReadChannel(Channels.newChannel(in)), allocator)
    +      .asInstanceOf[ArrowRecordBatch]  // throws IOException
       }
     
    +  /**
    +   * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches.
    +   */
       private[sql] def toDataFrame(
    -      payloadRDD: JavaRDD[Array[Byte]],
    +      arrowBatchRDD: JavaRDD[Array[Byte]],
           schemaString: String,
           sqlContext: SQLContext): DataFrame = {
    -    val rdd = payloadRDD.rdd.mapPartitions { iter =>
    +    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
    +    val timeZoneId = sqlContext.sessionState.conf.sessionLocalTimeZone
    +    val rdd = arrowBatchRDD.rdd.mapPartitions { iter =>
           val context = TaskContext.get()
    -      ArrowConverters.fromPayloadIterator(iter.map(new ArrowPayload(_)), context)
    +      ArrowConverters.fromBatchIterator(iter, schema, timeZoneId, context)
         }
    -    val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
         sqlContext.internalCreateDataFrame(rdd, schema)
       }
    +
    +  /**
    +   * Read a file as an Arrow stream and parallelize as an RDD of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def readArrowStreamFromFile(
    +      sqlContext: SQLContext,
    +      filename: String): JavaRDD[Array[Byte]] = {
    +    val fileStream = new FileInputStream(filename)
    +    try {
    +      // Create array so that we can safely close the file
    +      val batches = getBatchesFromStream(fileStream.getChannel).toArray
    +      // Parallelize the record batches to create an RDD
    +      JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length))
    +    } finally {
    +      fileStream.close()
    +    }
    +  }
    +
    +  /**
    +   * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches.
    +   */
    +  private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = {
    +
    +    // TODO: this could be moved to Arrow
    +    def readMessageLength(in: ReadChannel): Int = {
    +      val buffer = ByteBuffer.allocate(4)
    +      if (in.readFully(buffer) != 4) {
    +        return 0
    +      }
    +      MessageSerializer.bytesToInt(buffer.array())
    +    }
    +
    +    // TODO: this could be moved to Arrow
    +    def loadMessage(in: ReadChannel, messageLength: Int, buffer: ByteBuffer): Message = {
    +      if (in.readFully(buffer) != messageLength) {
    +        throw new java.io.IOException(
    +          "Unexpected end of stream trying to read message.")
    +      }
    +      buffer.rewind()
    +      Message.getRootAsMessage(buffer)
    +    }
    +
    +
    +    // Create an iterator to get each serialized ArrowRecordBatch from a stream
    +    new Iterator[Array[Byte]] {
    +      val inputChannel = new ReadChannel(in)
    +      var batch: Array[Byte] = readNextBatch()
    +
    +      override def hasNext: Boolean = batch != null
    +
    +      override def next(): Array[Byte] = {
    +        val prevBatch = batch
    +        batch = readNextBatch()
    +        prevBatch
    +      }
    +
    +      def readNextBatch(): Array[Byte] = {
    +        val messageLength = readMessageLength(inputChannel)
    +        if (messageLength == 0) {
    +          return null
    +        }
    +
    +        val buffer = ByteBuffer.allocate(messageLength)
    +        val msg = loadMessage(inputChannel, messageLength, buffer)
    --- End diff --
    
    Can we use `MessageChannelReader.readNextMessage()`?


---

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


[GitHub] spark issue #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream format ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    **[Test build #93137 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93137/testReport)** for PR 21546 at commit [`1308169`](https://github.com/apache/spark/commit/1308169ca7ea44395436e7620a6529a2c5f746cb).


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    retest this please


---

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


[GitHub] spark issue #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream format for c...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21546
  
    Yup, since `spark.sql.execution.arrow.enabled` is an experimental feature, we could just turn this off if there are critical bugs found later after the release.


---

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


[GitHub] spark pull request #21546: [SPARK-23030][SQL][PYTHON] Use Arrow stream forma...

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

    https://github.com/apache/spark/pull/21546#discussion_r204961977
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -184,27 +184,67 @@ def loads(self, obj):
             raise NotImplementedError
     
     
    -class ArrowSerializer(FramedSerializer):
    +class BatchOrderSerializer(Serializer):
    --- End diff --
    
    Since you verifies the performance difference is trivial, I don't think it's a hard requirement to merge this though. At least, I would just push this in.


---

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


[GitHub] spark pull request #21546: [WIP][SPARK-23030][SQL][PYTHON] Use Arrow stream ...

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

    https://github.com/apache/spark/pull/21546#discussion_r194948874
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
    @@ -3236,13 +3236,49 @@ class Dataset[T] private[sql](
       }
     
       /**
    -   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
    +   * Collect a Dataset as Arrow batches and serve stream to PySpark.
        */
       private[sql] def collectAsArrowToPython(): Array[Any] = {
    +    val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone
    +
         withAction("collectAsArrowToPython", queryExecution) { plan =>
    -      val iter: Iterator[Array[Byte]] =
    -        toArrowPayload(plan).collect().iterator.map(_.asPythonSerializable)
    -      PythonRDD.serveIterator(iter, "serve-Arrow")
    +      PythonRDD.serveToStream("serve-Arrow") { out =>
    +        val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId)
    +        val arrowBatchRdd = getArrowBatchRdd(plan)
    +        val numPartitions = arrowBatchRdd.partitions.length
    +
    +        // Store collection results for worst case of 1 to N-1 partitions
    +        val results = new Array[Array[Array[Byte]]](numPartitions - 1)
    +        var lastIndex = -1  // index of last partition written
    +
    +        // Handler to eagerly write partitions to Python in order
    +        def handlePartitionBatches(index: Int, arrowBatches: Array[Array[Byte]]): Unit = {
    +          // If result is from next partition in order
    +          if (index - 1 == lastIndex) {
    +            batchWriter.writeBatches(arrowBatches.iterator)
    +            lastIndex += 1
    +            // Write stored partitions that come next in order
    +            while (lastIndex < results.length && results(lastIndex) != null) {
    +              batchWriter.writeBatches(results(lastIndex).iterator)
    +              results(lastIndex) = null
    +              lastIndex += 1
    +            }
    +            // After last batch, end the stream
    +            if (lastIndex == results.length) {
    +              batchWriter.end()
    +            }
    +          } else {
    +            // Store partitions received out of order
    +            results(index - 1) = arrowBatches
    +          }
    +        }
    +
    +        sparkSession.sparkContext.runJob(
    +          arrowBatchRdd,
    +          (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
    +          0 until numPartitions,
    +          handlePartitionBatches)
    --- End diff --
    
    Instead of collecting partitions back at once and holding out of order partitions in driver waiting for partitions in order, is it better to incrementally run job on partitions in order and send streams to python side? So we don't need to hold out of order partitions in driver.


---

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