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

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-22 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r123654149
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -2620,6 +2629,74 @@ def range_frame_match():
 
 importlib.reload(window)
 
+
+@unittest.skipIf(not _have_arrow, "Arrow not installed")
+class ArrowTests(ReusedPySparkTestCase):
+
+@classmethod
+def setUpClass(cls):
+ReusedPySparkTestCase.setUpClass()
+cls.spark = SparkSession(cls.sc)
+cls.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+cls.schema = StructType([
+StructField("1_str_t", StringType(), True),
+StructField("2_int_t", IntegerType(), True),
+StructField("3_long_t", LongType(), True),
+StructField("4_float_t", FloatType(), True),
+StructField("5_double_t", DoubleType(), True)])
+cls.data = [("a", 1, 10, 0.2, 2.0),
+("b", 2, 20, 0.4, 4.0),
+("c", 3, 30, 0.8, 6.0)]
+
+def assertFramesEqual(self, df_with_arrow, df_without):
+msg = ("DataFrame from Arrow is not equal" +
+   ("\n\nWith Arrow:\n%s\n%s" % (df_with_arrow, 
df_with_arrow.dtypes)) +
+   ("\n\nWithout:\n%s\n%s" % (df_without, df_without.dtypes)))
+self.assertTrue(df_without.equals(df_with_arrow), msg=msg)
+
+def test_unsupported_datatype(self):
+schema = StructType([StructField("array", ArrayType(IntegerType(), 
False), True)])
+df = self.spark.createDataFrame([([1, 2, 3],)], schema=schema)
+with QuietTest(self.sc):
+self.assertRaises(Exception, lambda: df.toPandas())
+
+def test_null_conversion(self):
+df_null = self.spark.createDataFrame([tuple([None for _ in 
range(len(self.data[0]))])] +
+ self.data)
+pdf = df_null.toPandas()
+null_counts = pdf.isnull().sum().tolist()
+self.assertTrue(all([c == 1 for c in null_counts]))
+
+def test_toPandas_arrow_toggle(self):
+df = self.spark.createDataFrame(self.data, schema=self.schema)
+self.spark.conf.set("spark.sql.execution.arrow.enable", "false")
+pdf = df.toPandas()
+self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+pdf_arrow = df.toPandas()
+self.assertFramesEqual(pdf_arrow, pdf)
+
+def test_pandas_round_trip(self):
+import pandas as pd
+import numpy as np
+data_dict = {}
+for j, name in enumerate(self.schema.names):
+data_dict[name] = [self.data[i][j] for i in 
range(len(self.data))]
+# need to convert these to numpy types first
--- End diff --

is this still needed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-20 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r123128861
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -2473,6 +2482,76 @@ def range_frame_match():
 
 importlib.reload(window)
 
+
+@unittest.skipIf(not _have_arrow, "Arrow not installed")
+class ArrowTests(ReusedPySparkTestCase):
+
+@classmethod
+def setUpClass(cls):
+ReusedPySparkTestCase.setUpClass()
+cls.spark = SparkSession(cls.sc)
+cls.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+cls.schema = StructType([
+StructField("1_str_t", StringType(), True),
+StructField("2_int_t", IntegerType(), True),
+StructField("3_long_t", LongType(), True),
+StructField("4_float_t", FloatType(), True),
+StructField("5_double_t", DoubleType(), True)])
+cls.data = [("a", 1, 10, 0.2, 2.0),
+("b", 2, 20, 0.4, 4.0),
+("c", 3, 30, 0.8, 6.0)]
+
+def assertFramesEqual(self, df_with_arrow, df_without):
+msg = ("DataFrame from Arrow is not equal" +
+   ("\n\nWith Arrow:\n%s\n%s" % (df_with_arrow, 
df_with_arrow.dtypes)) +
+   ("\n\nWithout:\n%s\n%s" % (df_without, df_without.dtypes)))
+self.assertTrue(df_without.equals(df_with_arrow), msg=msg)
+
+def test_unsupported_datatype(self):
+schema = StructType([StructField("array", ArrayType(IntegerType(), 
False), True)])
+df = self.spark.createDataFrame([([1, 2, 3],)], schema=schema)
+with QuietTest(self.sc):
+self.assertRaises(Exception, lambda: df.toPandas())
+
+def test_null_conversion(self):
+df_null = self.spark.createDataFrame([tuple([None for _ in 
range(len(self.data[0]))])] +
+ self.data)
+pdf = df_null.toPandas()
+null_counts = pdf.isnull().sum().tolist()
+self.assertTrue(all([c == 1 for c in null_counts]))
+
+def test_toPandas_arrow_toggle(self):
+df = self.spark.createDataFrame(self.data, schema=self.schema)
+# NOTE - toPandas() without pyarrow will infer standard python 
data types
--- End diff --

makes sense, it's kind of a bug, maybe we should fix it for the non-arrow 
code path.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-20 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r123124184
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,429 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark and served 
to a Python process.
+ */
+private[sql] class ArrowPayload private[arrow] (payload: Array[Byte]) 
extends Serializable {
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as a type that can be served to Python.
+   */
+  def asPythonSerializable: Array[Byte] = payload
+}
+
+private[sql] object ArrowPayload {
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def apply(
+  batch: ArrowRecordBatch,
+  schema: StructType,
+  allocator: BufferAllocator): ArrowPayload = {
+new ArrowPayload(ArrowConverters.batchToByteArray(batch, schema, 
allocator))
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null

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

2017-06-20 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r123122759
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,429 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark and served 
to a Python process.
+ */
+private[sql] class ArrowPayload private[arrow] (payload: Array[Byte]) 
extends Serializable {
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as a type that can be served to Python.
+   */
+  def asPythonSerializable: Array[Byte] = payload
+}
+
+private[sql] object ArrowPayload {
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def apply(
+  batch: ArrowRecordBatch,
+  schema: StructType,
+  allocator: BufferAllocator): ArrowPayload = {
+new ArrowPayload(ArrowConverters.batchToByteArray(batch, schema, 
allocator))
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
--- End diff --

I was working on it but things were getting a little messy because of 
differences with time-zone usage in Spark (there have been some recent 
discussions on the mailing list about it).  So after discussing with @holdenk , 
we thought it would be best to keep this PR simple with support for primitive 
types and work on timestamps on a followup.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-20 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r123121124
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -2473,6 +2482,76 @@ def range_frame_match():
 
 importlib.reload(window)
 
+
+@unittest.skipIf(not _have_arrow, "Arrow not installed")
+class ArrowTests(ReusedPySparkTestCase):
+
+@classmethod
+def setUpClass(cls):
+ReusedPySparkTestCase.setUpClass()
+cls.spark = SparkSession(cls.sc)
+cls.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+cls.schema = StructType([
+StructField("1_str_t", StringType(), True),
+StructField("2_int_t", IntegerType(), True),
+StructField("3_long_t", LongType(), True),
+StructField("4_float_t", FloatType(), True),
+StructField("5_double_t", DoubleType(), True)])
+cls.data = [("a", 1, 10, 0.2, 2.0),
+("b", 2, 20, 0.4, 4.0),
+("c", 3, 30, 0.8, 6.0)]
+
+def assertFramesEqual(self, df_with_arrow, df_without):
+msg = ("DataFrame from Arrow is not equal" +
+   ("\n\nWith Arrow:\n%s\n%s" % (df_with_arrow, 
df_with_arrow.dtypes)) +
+   ("\n\nWithout:\n%s\n%s" % (df_without, df_without.dtypes)))
+self.assertTrue(df_without.equals(df_with_arrow), msg=msg)
+
+def test_unsupported_datatype(self):
+schema = StructType([StructField("array", ArrayType(IntegerType(), 
False), True)])
+df = self.spark.createDataFrame([([1, 2, 3],)], schema=schema)
+with QuietTest(self.sc):
+self.assertRaises(Exception, lambda: df.toPandas())
+
+def test_null_conversion(self):
+df_null = self.spark.createDataFrame([tuple([None for _ in 
range(len(self.data[0]))])] +
+ self.data)
+pdf = df_null.toPandas()
+null_counts = pdf.isnull().sum().tolist()
+self.assertTrue(all([c == 1 for c in null_counts]))
+
+def test_toPandas_arrow_toggle(self):
+df = self.spark.createDataFrame(self.data, schema=self.schema)
+# NOTE - toPandas() without pyarrow will infer standard python 
data types
--- End diff --

What I meant by this is if your Spark DataFrame has a column of 
`IntegerType` and `toPandas` is called without using Arrow, then Spark will 
pickle the data to standard python (python only has a single int type).  Then a 
pandas DataFrame is created using `from_records` with the python ints.  Calling 
this constructor forces pandas to infer the data type and it will choose 
`int64`.  With Arrow, the Spark schema is carried over, so pandas will know the 
correct data type is `int32`.  So if this test were to compare columns of 
`IntegerType` or `FloatType` with/without Arrow then the schema's would not 
match because of this.  The data is still the same, but I would say the schema 
is now correct with arrow.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-20 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122931371
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,429 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark and served 
to a Python process.
+ */
+private[sql] class ArrowPayload private[arrow] (payload: Array[Byte]) 
extends Serializable {
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as a type that can be served to Python.
+   */
+  def asPythonSerializable: Array[Byte] = payload
+}
+
+private[sql] object ArrowPayload {
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def apply(
+  batch: ArrowRecordBatch,
+  schema: StructType,
+  allocator: BufferAllocator): ArrowPayload = {
+new ArrowPayload(ArrowConverters.batchToByteArray(batch, schema, 
allocator))
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
--- End diff --

we do have timestamp writer, why we not support it here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-20 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122929684
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,429 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark and served 
to a Python process.
+ */
+private[sql] class ArrowPayload private[arrow] (payload: Array[Byte]) 
extends Serializable {
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as a type that can be served to Python.
+   */
+  def asPythonSerializable: Array[Byte] = payload
+}
+
+private[sql] object ArrowPayload {
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def apply(
+  batch: ArrowRecordBatch,
+  schema: StructType,
+  allocator: BufferAllocator): ArrowPayload = {
+new ArrowPayload(ArrowConverters.batchToByteArray(batch, schema, 
allocator))
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
 

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

2017-06-20 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122929031
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
--- End diff --

Spark doesn't always consume up the iterator, e.g. in a `LIMIT n` query. 
Usually we should register the clean up stuff in 
`TaskContext.addTaskCompletionListener`.

Let's address it in follow-up


---
If your project is set up for it, you can reply to this 

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

2017-06-20 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122927226
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2853,4 +2864,13 @@ class Dataset[T] private[sql](
   Dataset(sparkSession, logicalPlan)
 }
   }
+
+  /** Convert to an RDD of ArrowPayload byte arrays */
+  private[sql] def toArrowPayload: RDD[ArrowPayload] = {
+val schema_captured = this.schema
--- End diff --

nit: `schemaCaptured` is more java-style


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-20 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122925584
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -2473,6 +2482,76 @@ def range_frame_match():
 
 importlib.reload(window)
 
+
+@unittest.skipIf(not _have_arrow, "Arrow not installed")
+class ArrowTests(ReusedPySparkTestCase):
+
+@classmethod
+def setUpClass(cls):
+ReusedPySparkTestCase.setUpClass()
+cls.spark = SparkSession(cls.sc)
+cls.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+cls.schema = StructType([
+StructField("1_str_t", StringType(), True),
+StructField("2_int_t", IntegerType(), True),
+StructField("3_long_t", LongType(), True),
+StructField("4_float_t", FloatType(), True),
+StructField("5_double_t", DoubleType(), True)])
+cls.data = [("a", 1, 10, 0.2, 2.0),
+("b", 2, 20, 0.4, 4.0),
+("c", 3, 30, 0.8, 6.0)]
+
+def assertFramesEqual(self, df_with_arrow, df_without):
+msg = ("DataFrame from Arrow is not equal" +
+   ("\n\nWith Arrow:\n%s\n%s" % (df_with_arrow, 
df_with_arrow.dtypes)) +
+   ("\n\nWithout:\n%s\n%s" % (df_without, df_without.dtypes)))
+self.assertTrue(df_without.equals(df_with_arrow), msg=msg)
+
+def test_unsupported_datatype(self):
+schema = StructType([StructField("array", ArrayType(IntegerType(), 
False), True)])
+df = self.spark.createDataFrame([([1, 2, 3],)], schema=schema)
+with QuietTest(self.sc):
+self.assertRaises(Exception, lambda: df.toPandas())
+
+def test_null_conversion(self):
+df_null = self.spark.createDataFrame([tuple([None for _ in 
range(len(self.data[0]))])] +
+ self.data)
+pdf = df_null.toPandas()
+null_counts = pdf.isnull().sum().tolist()
+self.assertTrue(all([c == 1 for c in null_counts]))
+
+def test_toPandas_arrow_toggle(self):
+df = self.spark.createDataFrame(self.data, schema=self.schema)
+# NOTE - toPandas() without pyarrow will infer standard python 
data types
--- End diff --

oh, do you mean with the arrow optimization, we will have a different 
result for int and float?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-19 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122765858
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -0,0 +1,1218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.arrow
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.memory.RootAllocator
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayloads = indexData.toArrowPayload.collect()
+assert(arrowPayloads.nonEmpty)
+assert(arrowPayloads.length == indexData.rdd.getNumPartitions)
+val allocator = new RootAllocator(Long.MaxValue)
+val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator))
+val rowCount = arrowRecordBatches.map(_.getLength).sum
+assert(rowCount === indexData.count())
+arrowRecordBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowRecordBatches.foreach(_.close())
+allocator.close()
+  }
+
+  test("short conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "a_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : false,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |}, {
+ |  "name" : "b_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 6,
+ |"columns" : [ {
+ |  "name" : "a_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ],
+ |  "DATA" : [ 1, -1, 2, -2, 32767, -32768 ]
+ |}, {
+ |  "name" : "b_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ],
+ |  "DATA" : [ 1, 0, 0, -2, 0, -32768 ]
+ |} ]
+ 

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

2017-06-16 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122488149
  
--- Diff: python/pyspark/sql/dataframe.py ---
@@ -1648,8 +1650,30 @@ def toPandas(self):
 02  Alice
 15Bob
 """
-import pandas as pd
-return pd.DataFrame.from_records(self.collect(), 
columns=self.columns)
+if self.sql_ctx.getConf("spark.sql.execution.arrow.enable", 
"false").lower() == "true":
+try:
+import pyarrow
+tables = self._collectAsArrow()
+table = pyarrow.concat_tables(tables)
--- End diff --

Good point @leifwalsh , I have a test for an empty DataFrame in Java and a 
test for empty partition in Python, but there should one for this case also and 
it should definitely handle the error.  I'll put that in the next update.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread leifwalsh
Github user leifwalsh commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122359928
  
--- Diff: python/pyspark/sql/dataframe.py ---
@@ -1648,8 +1650,30 @@ def toPandas(self):
 02  Alice
 15Bob
 """
-import pandas as pd
-return pd.DataFrame.from_records(self.collect(), 
columns=self.columns)
+if self.sql_ctx.getConf("spark.sql.execution.arrow.enable", 
"false").lower() == "true":
+try:
+import pyarrow
+tables = self._collectAsArrow()
+table = pyarrow.concat_tables(tables)
--- End diff --

If tables is an empty list (e.g. if you load a dataset, filter the whole 
thing, and produce zero rows), `pyarrow.concat_tables` raises an exception 
rather than producing an empty table.  This should probably be fixed in arrow 
(cc @wesm) but we should be defensive here.  Probably should try to produce a 
`DataFrame` with the right schema but no rows if possible.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122359743
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
--- End diff --

sounds good


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122359727
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -0,0 +1,1218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.arrow
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.memory.RootAllocator
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayloads = indexData.toArrowPayload.collect()
+assert(arrowPayloads.nonEmpty)
+assert(arrowPayloads.length == indexData.rdd.getNumPartitions)
+val allocator = new RootAllocator(Long.MaxValue)
+val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator))
+val rowCount = arrowRecordBatches.map(_.getLength).sum
+assert(rowCount === indexData.count())
+arrowRecordBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowRecordBatches.foreach(_.close())
+allocator.close()
+  }
+
+  test("short conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "a_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : false,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |}, {
+ |  "name" : "b_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 6,
+ |"columns" : [ {
+ |  "name" : "a_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ],
+ |  "DATA" : [ 1, -1, 2, -2, 32767, -32768 ]
+ |}, {
+ |  "name" : "b_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ],
+ |  "DATA" : [ 1, 0, 0, -2, 0, -32768 ]
+ |} ]
+   

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

2017-06-15 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122359492
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+ 

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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122326641
  
--- Diff: dev/run-pip-tests ---
@@ -120,6 +122,10 @@ for python in "${PYTHON_EXECS[@]}"; do
 python "$FWDIR"/dev/pip-sanity-check.py
 echo "Run the tests for context.py"
 python "$FWDIR"/python/pyspark/context.py
+if [ -n "$TEST_PYARROW" ]; then
+  echo "Run tests for pyarrow"
+  SPARK_TESTING=1 "$FWDIR"/bin/pyspark pyspark.sql.tests ArrowTests
--- End diff --

Yeah, this will definitely be removed at some point.  I was working with 
@holdenk to set this up as a temporary way to get the Python Arrow tests to 
run.  I'll look into using the pip packages and see if that can be used instead 
of this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122326187
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+   

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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122326108
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+   

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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122305824
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -0,0 +1,1218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.arrow
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.memory.RootAllocator
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayloads = indexData.toArrowPayload.collect()
+assert(arrowPayloads.nonEmpty)
+assert(arrowPayloads.length == indexData.rdd.getNumPartitions)
+val allocator = new RootAllocator(Long.MaxValue)
+val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator))
+val rowCount = arrowRecordBatches.map(_.getLength).sum
+assert(rowCount === indexData.count())
+arrowRecordBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowRecordBatches.foreach(_.close())
+allocator.close()
+  }
+
+  test("short conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "a_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : false,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |}, {
+ |  "name" : "b_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 6,
+ |"columns" : [ {
+ |  "name" : "a_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ],
+ |  "DATA" : [ 1, -1, 2, -2, 32767, -32768 ]
+ |}, {
+ |  "name" : "b_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ],
+ |  "DATA" : [ 1, 0, 0, -2, 0, -32768 ]
+ |} ]
+ 

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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122305403
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -0,0 +1,1218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.arrow
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.memory.RootAllocator
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayloads = indexData.toArrowPayload.collect()
+assert(arrowPayloads.nonEmpty)
+assert(arrowPayloads.length == indexData.rdd.getNumPartitions)
+val allocator = new RootAllocator(Long.MaxValue)
+val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator))
+val rowCount = arrowRecordBatches.map(_.getLength).sum
+assert(rowCount === indexData.count())
+arrowRecordBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowRecordBatches.foreach(_.close())
+allocator.close()
+  }
+
+  test("short conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "a_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : false,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |}, {
+ |  "name" : "b_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 6,
+ |"columns" : [ {
+ |  "name" : "a_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ],
+ |  "DATA" : [ 1, -1, 2, -2, 32767, -32768 ]
+ |}, {
+ |  "name" : "b_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ],
+ |  "DATA" : [ 1, 0, 0, -2, 0, -32768 ]
+ |} ]
+ 

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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122288797
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
--- End diff --

I could move this work to a `ArrowPayload` object in `apply()` if that is 
preferable?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122288458
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
--- End diff --

Yes, this is safe.  It is probably not the most ideal usage, but is the 
least invasive to Spark.  It would be better (in a debugging sense - easier to 
identify issues) if there was a single root allocator in the JVM that could 
create child 

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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122287248
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
--- End diff --

I'm not sure of the difference, but I'll look into it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122284356
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
--- End diff --

True it's currently only used for test and will be needed in followup work. 
 It could be moved to test, but since these are package private, is that really 
necessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122282802
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -2473,6 +2482,69 @@ def range_frame_match():
 
 importlib.reload(window)
 
+
+@unittest.skipIf(not _have_arrow, "Arrow not installed")
+class ArrowTests(ReusedPySparkTestCase):
+
+@classmethod
+def setUpClass(cls):
+ReusedPySparkTestCase.setUpClass()
+cls.spark = SparkSession(cls.sc)
+cls.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+cls.schema = StructType([
+StructField("1_str_t", StringType(), True),
+StructField("2_int_t", IntegerType(), True),
+StructField("3_long_t", LongType(), True),
+StructField("4_float_t", FloatType(), True),
+StructField("5_double_t", DoubleType(), True)])
+cls.data = [("a", 1, 10, 0.2, 2.0),
+("b", 2, 20, 0.4, 4.0),
+("c", 3, 30, 0.8, 6.0)]
+
+def assertFramesEqual(self, df_with_arrow, df_without):
+msg = ("DataFrame from Arrow is not equal" +
+   ("\n\nWith Arrow:\n%s\n%s" % (df_with_arrow, 
df_with_arrow.dtypes)) +
+   ("\n\nWithout:\n%s\n%s" % (df_without, df_without.dtypes)))
+self.assertTrue(df_without.equals(df_with_arrow), msg=msg)
+
+def test_unsupported_datatype(self):
+schema = StructType([StructField("array", ArrayType(IntegerType(), 
False), True)])
+df = self.spark.createDataFrame([([1, 2, 3],)], schema=schema)
+with QuietTest(self.sc):
+self.assertRaises(Exception, lambda: df.toPandas())
+
+def test_null_conversion(self):
+df_null = self.spark.createDataFrame([tuple([None for _ in 
range(len(self.data[0]))])] +
+ self.data)
+pdf = df_null.toPandas()
+null_counts = pdf.isnull().sum().tolist()
+self.assertTrue(all([c == 1 for c in null_counts]))
+
+def test_toPandas_arrow_toggle(self):
+df = self.spark.createDataFrame(self.data, schema=self.schema)
+# NOTE - toPandas() without pyarrow will infer standard python 
data types
+df_sel = df.select("1_str_t", "3_long_t", "5_double_t")
+self.spark.conf.set("spark.sql.execution.arrow.enable", "false")
+pdf = df_sel.toPandas()
+self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+pdf_arrow = df_sel.toPandas()
+self.assertFramesEqual(pdf_arrow, pdf)
+
+def test_pandas_round_trip(self):
+import pandas as pd
+import numpy as np
+data_dict = {}
+for j, name in enumerate(self.schema.names):
+data_dict[name] = [self.data[i][j] for i in 
range(len(self.data))]
+# need to convert these to numpy types first
+data_dict["2_int_t"] = np.int32(data_dict["2_int_t"])
+data_dict["4_float_t"] = np.float32(data_dict["4_float_t"])
+pdf = pd.DataFrame(data=data_dict)
+df = self.spark.createDataFrame(self.data, schema=self.schema)
+pdf_arrow = df.toPandas()
--- End diff --

that's done in the `setUpClass` so it's already enabled for all tests.  
There is a test where it's toggled, off then back on though to test that 
behavior.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122282259
  
--- Diff: python/pyspark/sql/dataframe.py ---
@@ -1648,8 +1650,30 @@ def toPandas(self):
 02  Alice
 15Bob
 """
-import pandas as pd
-return pd.DataFrame.from_records(self.collect(), 
columns=self.columns)
+if self.sql_ctx.getConf("spark.sql.execution.arrow.enable", 
"false").lower() == "true":
+try:
+import pyarrow
+tables = self._collectAsArrow()
+table = pyarrow.concat_tables(tables)
+return table.to_pandas()
--- End diff --

`toPandas()` is a collect operation on the driver, so all data will have to 
be present before the pandas.DataFrame can be made regardless.  As @leifwalsh 
pointed out, the batch size limiting is to benefit the JVM running the 
executor, which might have different limitations that the driver.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122280237
  
--- Diff: python/pyspark/serializers.py ---
@@ -182,6 +182,23 @@ def loads(self, obj):
 raise NotImplementedError
 
 
+class ArrowSerializer(FramedSerializer):
+"""
+Serializes an Arrow stream.
+"""
+
+def dumps(self, obj):
+raise NotImplementedError
+
+def loads(self, obj):
+import pyarrow as pa
+reader = pa.RecordBatchFileReader(pa.BufferReader(obj))
+return reader.read_all()
--- End diff --

This will read all batches in a framed byte array from a stream and return. 
 The stream can have multiple framed byte arrays, so it repeats until end of 
stream.  

How many batches this reads depends on how it serialized.  When calling 
`toPandas()`, it collects all batches and then serializes each one to Python as 
an iterator.  So in this case, `reader.read_all()` will read 1 batch at a time. 
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r122273848
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+   

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

2017-06-14 Thread leifwalsh
Github user leifwalsh commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121922567
  
--- Diff: python/pyspark/sql/dataframe.py ---
@@ -1648,8 +1650,30 @@ def toPandas(self):
 02  Alice
 15Bob
 """
-import pandas as pd
-return pd.DataFrame.from_records(self.collect(), 
columns=self.columns)
+if self.sql_ctx.getConf("spark.sql.execution.arrow.enable", 
"false").lower() == "true":
+try:
+import pyarrow
+tables = self._collectAsArrow()
+table = pyarrow.concat_tables(tables)
+return table.to_pandas()
--- End diff --

The JVM may have a heap size limit lower than the memory available on the 
driver, so a batch size can help the driver stay under that limit even if the 
collected data are larger in memory on the python size than the JVM's max heap. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-14 Thread leifwalsh
Github user leifwalsh commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121922149
  
--- Diff: dev/run-pip-tests ---
@@ -120,6 +122,10 @@ for python in "${PYTHON_EXECS[@]}"; do
 python "$FWDIR"/dev/pip-sanity-check.py
 echo "Run the tests for context.py"
 python "$FWDIR"/python/pyspark/context.py
+if [ -n "$TEST_PYARROW" ]; then
+  echo "Run tests for pyarrow"
+  SPARK_TESTING=1 "$FWDIR"/bin/pyspark pyspark.sql.tests ArrowTests
--- End diff --

pyarrow has pip packages now, does that help? @BryanCutler I can't read 
your comment and understand why they required a conda env but maybe that's it? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-14 Thread leifwalsh
Github user leifwalsh commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121921861
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
--- End diff --

The read path will hopefully soon be used, as we also want to use arrow to 
move data from the executor to the python worker and back for UDF application. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-14 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121920656
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+ 

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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121834362
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -0,0 +1,1218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.arrow
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.memory.RootAllocator
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayloads = indexData.toArrowPayload.collect()
+assert(arrowPayloads.nonEmpty)
+assert(arrowPayloads.length == indexData.rdd.getNumPartitions)
+val allocator = new RootAllocator(Long.MaxValue)
+val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator))
+val rowCount = arrowRecordBatches.map(_.getLength).sum
+assert(rowCount === indexData.count())
+arrowRecordBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowRecordBatches.foreach(_.close())
+allocator.close()
+  }
+
+  test("short conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "a_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : false,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |}, {
+ |  "name" : "b_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 6,
+ |"columns" : [ {
+ |  "name" : "a_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ],
+ |  "DATA" : [ 1, -1, 2, -2, 32767, -32768 ]
+ |}, {
+ |  "name" : "b_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ],
+ |  "DATA" : [ 1, 0, 0, -2, 0, -32768 ]
+ |} ]
+   

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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121834255
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -0,0 +1,1218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.arrow
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.memory.RootAllocator
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayloads = indexData.toArrowPayload.collect()
+assert(arrowPayloads.nonEmpty)
+assert(arrowPayloads.length == indexData.rdd.getNumPartitions)
+val allocator = new RootAllocator(Long.MaxValue)
+val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator))
+val rowCount = arrowRecordBatches.map(_.getLength).sum
+assert(rowCount === indexData.count())
+arrowRecordBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowRecordBatches.foreach(_.close())
+allocator.close()
+  }
+
+  test("short conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "a_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : false,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |}, {
+ |  "name" : "b_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 6,
+ |"columns" : [ {
+ |  "name" : "a_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ],
+ |  "DATA" : [ 1, -1, 2, -2, 32767, -32768 ]
+ |}, {
+ |  "name" : "b_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ],
+ |  "DATA" : [ 1, 0, 0, -2, 0, -32768 ]
+ |} ]
+   

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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121834128
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -0,0 +1,1218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.arrow
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.memory.RootAllocator
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.{BinaryType, StructField, StructType}
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayloads = indexData.toArrowPayload.collect()
+assert(arrowPayloads.nonEmpty)
+assert(arrowPayloads.length == indexData.rdd.getNumPartitions)
+val allocator = new RootAllocator(Long.MaxValue)
+val arrowRecordBatches = arrowPayloads.map(_.loadBatch(allocator))
+val rowCount = arrowRecordBatches.map(_.getLength).sum
+assert(rowCount === indexData.count())
+arrowRecordBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowRecordBatches.foreach(_.close())
+allocator.close()
+  }
+
+  test("short conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "a_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : false,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |}, {
+ |  "name" : "b_s",
+ |  "type" : {
+ |"name" : "int",
+ |"isSigned" : true,
+ |"bitWidth" : 16
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 16
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 6,
+ |"columns" : [ {
+ |  "name" : "a_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 1, 1, 1, 1, 1 ],
+ |  "DATA" : [ 1, -1, 2, -2, 32767, -32768 ]
+ |}, {
+ |  "name" : "b_s",
+ |  "count" : 6,
+ |  "VALIDITY" : [ 1, 0, 0, 1, 0, 1 ],
+ |  "DATA" : [ 1, 0, 0, -2, 0, -32768 ]
+ |} ]
+   

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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121833587
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
--- End diff --

if this is only used in test, can we move these code to the test file?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121832853
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+ 

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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121831805
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
--- End diff --

it's weird to do this heavy work in the constructor. If we remove 
`loadBatch`, I think we don't need this class and can just return byte array to 
the caller


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121831641
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
--- End diff --

`toPayloadIterator` will be called for each task, and multiple tasks may be 
run in the same executor(same JVM), which means, we may have multiple 
`RootAllocator` inside on JVM. Is this safe?


---
If your project is set up for it, you can reply to 

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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121831432
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
--- End diff --

do you know what's the difference between Arrow decimal and Spark decimal? 
In Spark, the decimal will be stored as int if precision is less than 9, and be 
stored as long if precision is less than 18, and be stored as byte array 
otherwise


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121831202
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
--- End diff --

Actually the read path is unnecessary, we only send data from JVM to python 
with Arrow format, but we never read it back. Can we remove it and add it back 
when we really need it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121830995
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -2473,6 +2482,69 @@ def range_frame_match():
 
 importlib.reload(window)
 
+
+@unittest.skipIf(not _have_arrow, "Arrow not installed")
+class ArrowTests(ReusedPySparkTestCase):
+
+@classmethod
+def setUpClass(cls):
+ReusedPySparkTestCase.setUpClass()
+cls.spark = SparkSession(cls.sc)
+cls.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+cls.schema = StructType([
+StructField("1_str_t", StringType(), True),
+StructField("2_int_t", IntegerType(), True),
+StructField("3_long_t", LongType(), True),
+StructField("4_float_t", FloatType(), True),
+StructField("5_double_t", DoubleType(), True)])
+cls.data = [("a", 1, 10, 0.2, 2.0),
+("b", 2, 20, 0.4, 4.0),
+("c", 3, 30, 0.8, 6.0)]
+
+def assertFramesEqual(self, df_with_arrow, df_without):
+msg = ("DataFrame from Arrow is not equal" +
+   ("\n\nWith Arrow:\n%s\n%s" % (df_with_arrow, 
df_with_arrow.dtypes)) +
+   ("\n\nWithout:\n%s\n%s" % (df_without, df_without.dtypes)))
+self.assertTrue(df_without.equals(df_with_arrow), msg=msg)
+
+def test_unsupported_datatype(self):
+schema = StructType([StructField("array", ArrayType(IntegerType(), 
False), True)])
+df = self.spark.createDataFrame([([1, 2, 3],)], schema=schema)
+with QuietTest(self.sc):
+self.assertRaises(Exception, lambda: df.toPandas())
+
+def test_null_conversion(self):
+df_null = self.spark.createDataFrame([tuple([None for _ in 
range(len(self.data[0]))])] +
+ self.data)
+pdf = df_null.toPandas()
+null_counts = pdf.isnull().sum().tolist()
+self.assertTrue(all([c == 1 for c in null_counts]))
+
+def test_toPandas_arrow_toggle(self):
+df = self.spark.createDataFrame(self.data, schema=self.schema)
+# NOTE - toPandas() without pyarrow will infer standard python 
data types
+df_sel = df.select("1_str_t", "3_long_t", "5_double_t")
+self.spark.conf.set("spark.sql.execution.arrow.enable", "false")
+pdf = df_sel.toPandas()
+self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
+pdf_arrow = df_sel.toPandas()
+self.assertFramesEqual(pdf_arrow, pdf)
+
+def test_pandas_round_trip(self):
+import pandas as pd
+import numpy as np
+data_dict = {}
+for j, name in enumerate(self.schema.names):
+data_dict[name] = [self.data[i][j] for i in 
range(len(self.data))]
+# need to convert these to numpy types first
+data_dict["2_int_t"] = np.int32(data_dict["2_int_t"])
+data_dict["4_float_t"] = np.float32(data_dict["4_float_t"])
+pdf = pd.DataFrame(data=data_dict)
+df = self.spark.createDataFrame(self.data, schema=self.schema)
+pdf_arrow = df.toPandas()
--- End diff --

call `self.spark.conf.set("spark.sql.execution.arrow.enable", "true")` 
before this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121830561
  
--- Diff: python/pyspark/sql/dataframe.py ---
@@ -1648,8 +1650,30 @@ def toPandas(self):
 02  Alice
 15Bob
 """
-import pandas as pd
-return pd.DataFrame.from_records(self.collect(), 
columns=self.columns)
+if self.sql_ctx.getConf("spark.sql.execution.arrow.enable", 
"false").lower() == "true":
+try:
+import pyarrow
+tables = self._collectAsArrow()
+table = pyarrow.concat_tables(tables)
+return table.to_pandas()
--- End diff --

so we have to load all the data to memory before creating pandas dataframe? 
Then it's useless to set a batch size limitation at JVM side...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121830292
  
--- Diff: python/pyspark/serializers.py ---
@@ -182,6 +182,23 @@ def loads(self, obj):
 raise NotImplementedError
 
 
+class ArrowSerializer(FramedSerializer):
+"""
+Serializes an Arrow stream.
+"""
+
+def dumps(self, obj):
+raise NotImplementedError
+
+def loads(self, obj):
+import pyarrow as pa
+reader = pa.RecordBatchFileReader(pa.BufferReader(obj))
+return reader.read_all()
--- End diff --

since we are sending multiple batches from JVM, does `reader.read_all()` 
wait for all the bacthes?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121830120
  
--- Diff: dev/run-pip-tests ---
@@ -120,6 +122,10 @@ for python in "${PYTHON_EXECS[@]}"; do
 python "$FWDIR"/dev/pip-sanity-check.py
 echo "Run the tests for context.py"
 python "$FWDIR"/python/pyspark/context.py
+if [ -n "$TEST_PYARROW" ]; then
+  echo "Run tests for pyarrow"
+  SPARK_TESTING=1 "$FWDIR"/bin/pyspark pyspark.sql.tests ArrowTests
--- End diff --

will we remove this at some time?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121815649
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+ 

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

2017-06-13 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121758634
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+   

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

2017-06-13 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121750946
  
--- Diff: pom.xml ---
@@ -1871,6 +1872,25 @@
 paranamer
 ${paranamer.version}
   
+  
+org.apache.arrow
--- End diff --

This is just the dependency management section in the main pom.  The only 
actual dependency is in spark-sql.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121729635
  
--- Diff: pom.xml ---
@@ -1871,6 +1872,25 @@
 paranamer
 ${paranamer.version}
   
+  
+org.apache.arrow
--- End diff --

i think this is just standard pom ..


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-13 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121614002
  
--- Diff: pom.xml ---
@@ -1871,6 +1872,25 @@
 paranamer
 ${paranamer.version}
   
+  
+org.apache.arrow
--- End diff --

why we add arrow dependency at root instead of only spark sql?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-06-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r121202886
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+ 

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

2017-05-10 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115802562
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
+_nextPayload = null
+  }
+}
+obj
+  }
+
+  private def convert(): ArrowPayload = {
+val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator)
+new ArrowPayload(batch, schema, _allocator)

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

2017-05-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115635722
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
+_nextPayload = null
+  }
+}
+obj
+  }
+
+  private def convert(): ArrowPayload = {
+val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator)
+new ArrowPayload(batch, schema, _allocator)
  

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

2017-05-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115635616
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,423 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.FloatingPointPrecision
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark.
+ */
+private[sql] class ArrowPayload(payload: Array[Byte]) extends Serializable 
{
+
+  /**
+   * Create an ArrowPayload from an ArrowRecordBatch and Spark schema.
+   */
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  /**
+   * Convert the ArrowPayload to an ArrowRecordBatch.
+   */
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(payload, allocator)
+  }
+
+  /**
+   * Get the ArrowPayload as an Array[Byte].
+   */
+  def toByteArray: Array[Byte] = payload
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload. Limit 
ArrowRecordBatch size in ArrowPayload
+   * by setting maxRecordsPerBatch or use 0 to fully consume rowIter.
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType,
+  maxRecordsPerBatch: Int): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+ 

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

2017-05-09 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115556273
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
--- End diff --

The definition of the payload internal storage is a little vague on purpose 
because it could change and store more than one batch, or in a different 
format.  I'll add some more comments so that it is a little more clear what the 
input/output to an `ArrowPayload` is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-05-09 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115554122
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
--- End diff --

I believe all exceptions that could be raised by Arrow are handled now.  
Task cancellation is a little different, I'm not sure it could be interrupted 
during this conversion so that a memory leak would occur, but I'll have to 
think a little more about it..


---

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

2017-05-09 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115552830
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
+_nextPayload = null
+  }
+}
+obj
+  }
+
+  private def convert(): ArrowPayload = {
+val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator)
+new ArrowPayload(batch, schema, _allocator)

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

2017-05-09 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115552398
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
+_nextPayload = null
+  }
+}
+obj
+  }
+
+  private def convert(): ArrowPayload = {
+val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator)
+new ArrowPayload(batch, schema, _allocator)

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

2017-05-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115527041
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
+_nextPayload = null
+  }
+}
+obj
+  }
+
+  private def convert(): ArrowPayload = {
+val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator)
+new ArrowPayload(batch, schema, _allocator)
  

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

2017-05-09 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r115525849
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
+_nextPayload = null
+  }
+}
+obj
+  }
+
+  private def convert(): ArrowPayload = {
+val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator)
+new ArrowPayload(batch, schema, _allocator)
  

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

2017-04-27 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113730387
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
+_nextPayload = null
+  }
+}
+obj
+  }
+
+  private def convert(): ArrowPayload = {
+val batch = internalRowIterToArrowBatch(rowIter, schema, 
_allocator)
+new ArrowPayload(batch, schema, _allocator)
 

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

2017-04-27 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113729309
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
--- End diff --

I think it would be helpful to document that batchBytes here is in "Arrow 
FIle Format" that contains exactly one arrow record batch


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-27 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113728646
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
--- End diff --

I think it's better to have root allocator and use child allocators from 
the root allocator when an allocator is needed. This way it will be easy to 
find memory leaks. Maybe @julienledem can chime in and suggest the best 
practice? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: 

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

2017-04-27 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113728111
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala
 ---
@@ -0,0 +1,396 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.execution.arrow
+
+import java.io.ByteArrayOutputStream
+import java.nio.channels.Channels
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BufferAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file._
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, 
TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, 
Schema}
+import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * Store Arrow data in a form that can be serialized by Spark
+ */
+private[sql] class ArrowPayload(val batchBytes: Array[Byte]) extends 
Serializable {
+
+  def this(batch: ArrowRecordBatch, schema: StructType, allocator: 
BufferAllocator) = {
+this(ArrowConverters.batchToByteArray(batch, schema, allocator))
+  }
+
+  def loadBatch(allocator: BufferAllocator): ArrowRecordBatch = {
+ArrowConverters.byteArrayToBatch(batchBytes, allocator)
+  }
+}
+
+private[sql] object ArrowConverters {
+
+  /**
+   * Map a Spark DataType to ArrowType.
+   */
+  private[arrow] def sparkTypeToArrowType(dataType: DataType): ArrowType = 
{
+dataType match {
+  case BooleanType => ArrowType.Bool.INSTANCE
+  case ShortType => new ArrowType.Int(8 * ShortType.defaultSize, true)
+  case IntegerType => new ArrowType.Int(8 * IntegerType.defaultSize, 
true)
+  case LongType => new ArrowType.Int(8 * LongType.defaultSize, true)
+  case FloatType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)
+  case DoubleType => new 
ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)
+  case ByteType => new ArrowType.Int(8, true)
+  case StringType => ArrowType.Utf8.INSTANCE
+  case BinaryType => ArrowType.Binary.INSTANCE
+  case _ => throw new UnsupportedOperationException(s"Unsupported data 
type: $dataType")
+}
+  }
+
+  /**
+   * Convert a Spark Dataset schema to Arrow schema.
+   */
+  private[arrow] def schemaToArrowSchema(schema: StructType): Schema = {
+val arrowFields = schema.fields.map { f =>
+  new Field(f.name, f.nullable, sparkTypeToArrowType(f.dataType), 
List.empty[Field].asJava)
+}
+new Schema(arrowFields.toList.asJava)
+  }
+
+  /**
+   * Maps Iterator from InternalRow to ArrowPayload
+   */
+  private[sql] def toPayloadIterator(
+  rowIter: Iterator[InternalRow],
+  schema: StructType): Iterator[ArrowPayload] = {
+new Iterator[ArrowPayload] {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+  private var _nextPayload = if (rowIter.nonEmpty) convert() else null
+
+  override def hasNext: Boolean = _nextPayload != null
+
+  override def next(): ArrowPayload = {
+val obj = _nextPayload
+if (hasNext) {
+  if (rowIter.hasNext) {
+_nextPayload = convert()
+  } else {
+_allocator.close()
--- End diff --

I think also need to handle exception / task cancellation here to free up 
the memory.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, 

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

2017-04-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113600926
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/ArrowConvertersSuite.scala ---
@@ -0,0 +1,568 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.json4s.jackson.JsonMethods._
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  private def collectAsArrow(df: DataFrame,
+ converter: Option[ArrowConverters] = None): 
ArrowPayload = {
+val cnvtr = converter.getOrElse(new ArrowConverters)
+val payloadByteArrays = df.toArrowPayloadBytes().collect()
+cnvtr.readPayloadByteArrays(payloadByteArrays)
+  }
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayload = collectAsArrow(indexData)
+assert(arrowPayload.nonEmpty)
+val arrowBatches = arrowPayload.toArray
+assert(arrowBatches.length == indexData.rdd.getNumPartitions)
+val rowCount = arrowBatches.map(batch => batch.getLength).sum
+assert(rowCount === indexData.count())
+arrowBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowBatches.foreach(batch => batch.close())
+  }
+
+  test("numeric type conversion") {
+collectAndValidate(indexData)
+collectAndValidate(shortData)
+collectAndValidate(intData)
+collectAndValidate(longData)
+collectAndValidate(floatData)
+collectAndValidate(doubleData)
+  }
+
+  test("mixed numeric type conversion") {
+collectAndValidate(mixedNumericData)
+  }
+
+  test("boolean type conversion") {
+collectAndValidate(boolData)
+  }
+
+  test("string type conversion") {
+collectAndValidate(stringData)
+  }
+
+  test("byte type conversion") {
+collectAndValidate(byteData)
+  }
+
+  ignore("timestamp conversion") {
+collectAndValidate(timestampData)
+  }
+
+  // TODO: Not currently supported in Arrow JSON reader
+  ignore("date conversion") {
+// collectAndValidate(dateTimeData)
+  }
+
+  // TODO: Not currently supported in Arrow JSON reader
+  ignore("binary type conversion") {
+// collectAndValidate(binaryData)
+  }
+
+  test("floating-point NaN") {
+collectAndValidate(floatNaNData)
+  }
+
+  test("partitioned DataFrame") {
+val converter = new ArrowConverters
+val schema = testData2.schema
+val arrowPayload = collectAsArrow(testData2, Some(converter))
+val arrowBatches = arrowPayload.toArray
+// NOTE: testData2 should have 2 partitions -> 2 arrow batches in 
payload
+assert(arrowBatches.length === 2)
+val pl1 = new ArrowStaticPayload(arrowBatches(0))
+val pl2 = new ArrowStaticPayload(arrowBatches(1))
+// Generate JSON files
+val a = List[Int](1, 1, 2, 2, 3, 3)
+val b = List[Int](1, 2, 1, 2, 1, 2)
+val 

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

2017-04-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113600552
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113600492
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113600390
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113600189
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-26 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r113600062
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
--- End diff --

No, I think this should be changed a little.  `ArrowPayload` is meant to 
encapsulate Arrow classes from the rest of Spark and wrap Arrow data to extend 
`serializable` to allow an `RDD[ArrowPayload]`.  I'll push an update that will 
clean this up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-20 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112382152
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/ArrowConvertersSuite.scala ---
@@ -0,0 +1,568 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.json4s.jackson.JsonMethods._
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  private def collectAsArrow(df: DataFrame,
+ converter: Option[ArrowConverters] = None): 
ArrowPayload = {
+val cnvtr = converter.getOrElse(new ArrowConverters)
+val payloadByteArrays = df.toArrowPayloadBytes().collect()
+cnvtr.readPayloadByteArrays(payloadByteArrays)
+  }
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayload = collectAsArrow(indexData)
+assert(arrowPayload.nonEmpty)
+val arrowBatches = arrowPayload.toArray
+assert(arrowBatches.length == indexData.rdd.getNumPartitions)
+val rowCount = arrowBatches.map(batch => batch.getLength).sum
+assert(rowCount === indexData.count())
+arrowBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowBatches.foreach(batch => batch.close())
+  }
+
+  test("numeric type conversion") {
+collectAndValidate(indexData)
+collectAndValidate(shortData)
+collectAndValidate(intData)
+collectAndValidate(longData)
+collectAndValidate(floatData)
+collectAndValidate(doubleData)
+  }
+
+  test("mixed numeric type conversion") {
+collectAndValidate(mixedNumericData)
+  }
+
+  test("boolean type conversion") {
+collectAndValidate(boolData)
+  }
+
+  test("string type conversion") {
+collectAndValidate(stringData)
+  }
+
+  test("byte type conversion") {
+collectAndValidate(byteData)
+  }
+
+  ignore("timestamp conversion") {
+collectAndValidate(timestampData)
+  }
+
+  // TODO: Not currently supported in Arrow JSON reader
+  ignore("date conversion") {
+// collectAndValidate(dateTimeData)
+  }
+
+  // TODO: Not currently supported in Arrow JSON reader
+  ignore("binary type conversion") {
+// collectAndValidate(binaryData)
+  }
+
+  test("floating-point NaN") {
+collectAndValidate(floatNaNData)
+  }
+
+  test("partitioned DataFrame") {
+val converter = new ArrowConverters
+val schema = testData2.schema
+val arrowPayload = collectAsArrow(testData2, Some(converter))
+val arrowBatches = arrowPayload.toArray
+// NOTE: testData2 should have 2 partitions -> 2 arrow batches in 
payload
+assert(arrowBatches.length === 2)
+val pl1 = new ArrowStaticPayload(arrowBatches(0))
+val pl2 = new ArrowStaticPayload(arrowBatches(1))
+// Generate JSON files
+val a = List[Int](1, 1, 2, 2, 3, 3)
+val b = List[Int](1, 2, 1, 2, 1, 2)
+val fields = 

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

2017-04-20 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112381608
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/ArrowConvertersSuite.scala ---
@@ -0,0 +1,568 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.sql.{Date, Timestamp}
+import java.text.SimpleDateFormat
+import java.util.Locale
+
+import com.google.common.io.Files
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.file.json.JsonFileReader
+import org.apache.arrow.vector.util.Validator
+import org.json4s.jackson.JsonMethods._
+import org.json4s.JsonAST._
+import org.json4s.JsonDSL._
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+
+
+class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll 
{
+  import testImplicits._
+
+  private var tempDataPath: String = _
+
+  private def collectAsArrow(df: DataFrame,
+ converter: Option[ArrowConverters] = None): 
ArrowPayload = {
+val cnvtr = converter.getOrElse(new ArrowConverters)
+val payloadByteArrays = df.toArrowPayloadBytes().collect()
+cnvtr.readPayloadByteArrays(payloadByteArrays)
+  }
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+tempDataPath = Utils.createTempDir(namePrefix = 
"arrow").getAbsolutePath
+  }
+
+  test("collect to arrow record batch") {
+val indexData = (1 to 6).toDF("i")
+val arrowPayload = collectAsArrow(indexData)
+assert(arrowPayload.nonEmpty)
+val arrowBatches = arrowPayload.toArray
+assert(arrowBatches.length == indexData.rdd.getNumPartitions)
+val rowCount = arrowBatches.map(batch => batch.getLength).sum
+assert(rowCount === indexData.count())
+arrowBatches.foreach(batch => assert(batch.getNodes.size() > 0))
+arrowBatches.foreach(batch => batch.close())
+  }
+
+  test("numeric type conversion") {
+collectAndValidate(indexData)
--- End diff --

separate these into different test cases, and please inline the data 
directly in each test case. It's pretty annoying to have to jump around.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-20 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112376143
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-20 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112376037
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-20 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112375921
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-20 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112375496
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-20 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112373858
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-20 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112373805
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112370906
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112370321
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
--- End diff --

why even have this function? just change the signature of 
ArrowConverters.internalRowIterToArrowBatch and call that directly. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

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

2017-04-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112368956
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
--- End diff --

will there be other payload types that are not ArrowStaticPayload?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112368367
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/ArrowConverters.scala ---
@@ -0,0 +1,432 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import java.io.ByteArrayOutputStream
+import java.nio.ByteBuffer
+import java.nio.channels.{Channels, SeekableByteChannel}
+
+import scala.collection.JavaConverters._
+
+import io.netty.buffer.ArrowBuf
+import org.apache.arrow.memory.{BaseAllocator, RootAllocator}
+import org.apache.arrow.vector._
+import org.apache.arrow.vector.BaseValueVector.BaseMutator
+import org.apache.arrow.vector.file.{ArrowReader, ArrowWriter}
+import org.apache.arrow.vector.schema.{ArrowFieldNode, ArrowRecordBatch}
+import org.apache.arrow.vector.types.{FloatingPointPrecision, TimeUnit}
+import org.apache.arrow.vector.types.pojo.{ArrowType, Field, Schema}
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.util.Utils
+
+
+/**
+ * ArrowReader requires a seekable byte channel.
+ * TODO: This is available in arrow-vector now with ARROW-615, to be 
included in 0.2.1 release
+ */
+private[sql] class ByteArrayReadableSeekableByteChannel(var byteArray: 
Array[Byte])
+  extends SeekableByteChannel {
+  var _position: Long = 0L
+
+  override def isOpen: Boolean = {
+byteArray != null
+  }
+
+  override def close(): Unit = {
+byteArray = null
+  }
+
+  override def read(dst: ByteBuffer): Int = {
+val remainingBuf = byteArray.length - _position
+val length = Math.min(dst.remaining(), remainingBuf).toInt
+dst.put(byteArray, _position.toInt, length)
+_position += length
+length
+  }
+
+  override def position(): Long = _position
+
+  override def position(newPosition: Long): SeekableByteChannel = {
+_position = newPosition.toLong
+this
+  }
+
+  override def size: Long = {
+byteArray.length.toLong
+  }
+
+  override def write(src: ByteBuffer): Int = {
+throw new UnsupportedOperationException("Read Only")
+  }
+
+  override def truncate(size: Long): SeekableByteChannel = {
+throw new UnsupportedOperationException("Read Only")
+  }
+}
+
+/**
+ * Intermediate data structure returned from Arrow conversions
+ */
+private[sql] abstract class ArrowPayload extends Iterator[ArrowRecordBatch]
+
+/**
+ * Build a payload from existing ArrowRecordBatches
+ */
+private[sql] class ArrowStaticPayload(batches: ArrowRecordBatch*) extends 
ArrowPayload {
+  private val iter = batches.iterator
+  override def next(): ArrowRecordBatch = iter.next()
+  override def hasNext: Boolean = iter.hasNext
+}
+
+/**
+ * Class that wraps an Arrow RootAllocator used in conversion
+ */
+private[sql] class ArrowConverters {
+  private val _allocator = new RootAllocator(Long.MaxValue)
+
+  private[sql] def allocator: RootAllocator = _allocator
+
+  /**
+   * Iterate over the rows and convert to an ArrowPayload, using 
RootAllocator from this class
+   */
+  def interalRowIterToPayload(rowIter: Iterator[InternalRow], schema: 
StructType): ArrowPayload = {
+val batch = ArrowConverters.internalRowIterToArrowBatch(rowIter, 
schema, _allocator)
+new ArrowStaticPayload(batch)
+  }
+
+  /**
+   * Read an Array of Arrow Record batches as byte Arrays into an 
ArrowPayload, using
+   * RootAllocator from this class
+   */
+  def readPayloadByteArrays(payloadByteArrays: Array[Array[Byte]]): 
ArrowPayload = {
+val batches = 
scala.collection.mutable.ArrayBuffer.empty[ArrowRecordBatch]
+var i = 0
+while (i < payloadByteArrays.length) {
+  val payloadBytes = payloadByteArrays(i)
+  val in = new 

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

2017-04-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112365872
  
--- Diff: python/pyspark/sql/dataframe.py ---
@@ -1635,21 +1636,49 @@ def toDF(self, *cols):
 return DataFrame(jdf, self.sql_ctx)
 
 @since(1.3)
-def toPandas(self):
-"""Returns the contents of this :class:`DataFrame` as Pandas 
``pandas.DataFrame``.
+def toPandas(self, useArrow=False):
+"""
+Returns the contents of this :class:`DataFrame` as Pandas 
``pandas.DataFrame``.
 
 This is only available if Pandas is installed and available.
 
+:param useArrow: Make use of Apache Arrow for conversion, pyarrow 
must be installed
+and available on the calling Python process (Experimental).
+
 .. note:: This method should only be used if the resulting 
Pandas's DataFrame is expected
 to be small, as all the data is loaded into the driver's 
memory.
 
+.. note:: Using pyarrow is experimental and currently supports the 
following data types:
--- End diff --

remove this after you moved the control to a SQLConf.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112365773
  
--- Diff: python/pyspark/sql/dataframe.py ---
@@ -1635,21 +1636,49 @@ def toDF(self, *cols):
 return DataFrame(jdf, self.sql_ctx)
 
 @since(1.3)
-def toPandas(self):
-"""Returns the contents of this :class:`DataFrame` as Pandas 
``pandas.DataFrame``.
+def toPandas(self, useArrow=False):
--- End diff --

rather than having a flag here, I'd add a SQLConf to control this.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r112365501
  
--- Diff: python/pyspark/serializers.py ---
@@ -182,6 +182,23 @@ def loads(self, obj):
 raise NotImplementedError
 
 
+class ArrowSerializer(FramedSerializer):
+"""
+Serializes an Arrow stream.
+"""
+
+def dumps(self, obj):
+raise NotImplementedError
+
+def loads(self, obj):
+from pyarrow import FileReader, BufferReader
--- End diff --

is there a way for us to package that in spark directly?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-13 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r111512686
  
--- Diff: dev/run-pip-tests ---
@@ -120,6 +122,10 @@ for python in "${PYTHON_EXECS[@]}"; do
 python "$FWDIR"/dev/pip-sanity-check.py
 echo "Run the tests for context.py"
 python "$FWDIR"/python/pyspark/context.py
+if [ -n "$TEST_PYARROW" ]; then
+  echo "Run tests for pyarrow"
+  SPARK_TESTING=1 "$FWDIR"/bin/pyspark pyspark.sql.tests ArrowTests
--- End diff --

This is just a temporary addition to make sure pyarrow tests run because 
they required a conda env, for now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-03 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r109547685
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2747,6 +2747,17 @@ class Dataset[T] private[sql](
 }
   }
 
+  /**
+   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
+   */
+  private[sql] def collectAsArrowToPython(): Int = {
+val payloadRdd = toArrowPayloadBytes()
+val payloadByteArrays = payloadRdd.collect()
--- End diff --

The conversion going on in `table.to_pandas()` is working on an already 
loaded table, but the Arrow Readers can read multiple batches of data and 
output a single table.  The issue is that pyspark serializers expects the data 
to be "framed" with the length so I can not send that directly to the Arrow 
Reader.  Even with `toLocalIteratorAndServer` I would have to read each batch 
of data on the driver, then combine.  It would be possible to write the 
"framed" stream another stream without the lengths, where it can then be then 
be read into a single table - but I'm not sure if that added complexity is 
worth it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-03 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r109519617
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2747,6 +2747,17 @@ class Dataset[T] private[sql](
 }
   }
 
+  /**
+   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
+   */
+  private[sql] def collectAsArrowToPython(): Int = {
+val payloadRdd = toArrowPayloadBytes()
+val payloadByteArrays = payloadRdd.collect()
--- End diff --

Right, but since it had a small effect on performance I thought it would be 
better to avoid for now.  I have also been experimenting working with the Arrow 
`StreamWriter` which does reduce memory usage and improves performance, but 
required some additional changes to Spark so I think it would be better to 
propose as a follow up PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-04-03 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r109517888
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2828,4 +2839,16 @@ class Dataset[T] private[sql](
   Dataset(sparkSession, logicalPlan)
 }
   }
+
+  /** Convert to an RDD of ArrowPayload byte arrays */
+  private[sql] def toArrowPayloadBytes(): RDD[Array[Byte]] = {
+val schema_captured = this.schema
+queryExecution.toRdd.mapPartitionsInternal { iter =>
+  val converter = new ArrowConverters
+  val payload = converter.interalRowIterToPayload(iter, 
schema_captured)
+  val payloadBytes = ArrowConverters.payloadToByteArray(payload, 
schema_captured)
--- End diff --

Would it be better to use a constant for number of elements?  If there are 
only a few columns then limiting to 1000 rows might be too little.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108835232
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2747,6 +2747,17 @@ class Dataset[T] private[sql](
 }
   }
 
+  /**
+   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
+   */
+  private[sql] def collectAsArrowToPython(): Int = {
+val payloadRdd = toArrowPayloadBytes()
+val payloadByteArrays = payloadRdd.collect()
--- End diff --

@BryanCutler Btw, it is not for performance gain I think. 
`toLocalIteratorAndServe` can avoid collect all data at once into the driver. 
So it may be good for the memory usage on the driver side.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread wesm
Github user wesm commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108834631
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2747,6 +2747,17 @@ class Dataset[T] private[sql](
 }
   }
 
+  /**
+   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
+   */
+  private[sql] def collectAsArrowToPython(): Int = {
+val payloadRdd = toArrowPayloadBytes()
+val payloadByteArrays = payloadRdd.collect()
--- End diff --

You can stream out payloads as they come into the driver (maybe this is 
already happening). We may be able to play with the StreamWriter to reduce the 
driver memory usage in a follow up patch


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108834618
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2828,4 +2839,16 @@ class Dataset[T] private[sql](
   Dataset(sparkSession, logicalPlan)
 }
   }
+
+  /** Convert to an RDD of ArrowPayload byte arrays */
+  private[sql] def toArrowPayloadBytes(): RDD[Array[Byte]] = {
+val schema_captured = this.schema
+queryExecution.toRdd.mapPartitionsInternal { iter =>
+  val converter = new ArrowConverters
+  val payload = converter.interalRowIterToPayload(iter, 
schema_captured)
+  val payloadBytes = ArrowConverters.payloadToByteArray(payload, 
schema_captured)
--- End diff --

Do you think we need a dedicated config for it? Or maybe a constant like 
1000 (rows)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108834139
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2747,6 +2747,17 @@ class Dataset[T] private[sql](
 }
   }
 
+  /**
+   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
+   */
+  private[sql] def collectAsArrowToPython(): Int = {
+val payloadRdd = toArrowPayloadBytes()
+val payloadByteArrays = payloadRdd.collect()
--- End diff --

Ok. As actually to make the Pandas's DataFrame, all data are needed to load 
into driver's memory, `toLocalIteratorAndServe` can't improve the memory usage 
in the end.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108833855
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -56,6 +56,15 @@
 from pyspark.sql.utils import AnalysisException, ParseException, 
IllegalArgumentException
 
 
+_have_arrow = False
+try:
+import pyarrow
+_have_arrow = True
--- End diff --

Maybe give the param doc string as exception message?

I.e., `To make use of Apache Arrow for conversion, pyarrow must be 
installed and available on the calling Python process (Experimental)`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108833416
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -56,6 +56,15 @@
 from pyspark.sql.utils import AnalysisException, ParseException, 
IllegalArgumentException
 
 
+_have_arrow = False
+try:
+import pyarrow
+_have_arrow = True
--- End diff --

I mean we should throw an exception when `useArrow` is used but no pyspark 
is installed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108746678
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2747,6 +2747,17 @@ class Dataset[T] private[sql](
 }
   }
 
+  /**
+   * Collect a Dataset as ArrowPayload byte arrays and serve to PySpark.
+   */
+  private[sql] def collectAsArrowToPython(): Int = {
+val payloadRdd = toArrowPayloadBytes()
+val payloadByteArrays = payloadRdd.collect()
--- End diff --

I did some experimenting with `toLocalIteratorAndServe` and did not see any 
performance gain - it was actually a little worse in local mode at least.  It 
might be worth checking out again though.  There are other ways to go about 
serving to python, but the current implementation seemed to be the least 
intrusive to Spark.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108744852
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -2828,4 +2839,16 @@ class Dataset[T] private[sql](
   Dataset(sparkSession, logicalPlan)
 }
   }
+
+  /** Convert to an RDD of ArrowPayload byte arrays */
+  private[sql] def toArrowPayloadBytes(): RDD[Array[Byte]] = {
+val schema_captured = this.schema
+queryExecution.toRdd.mapPartitionsInternal { iter =>
+  val converter = new ArrowConverters
+  val payload = converter.interalRowIterToPayload(iter, 
schema_captured)
+  val payloadBytes = ArrowConverters.payloadToByteArray(payload, 
schema_captured)
--- End diff --

Yes, that is a good idea.  It does support having multiple payloads so that 
would not be too difficult to change, but the only question is how big to make 
the groups?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



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

2017-03-29 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/15821#discussion_r108744456
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -56,6 +56,15 @@
 from pyspark.sql.utils import AnalysisException, ParseException, 
IllegalArgumentException
 
 
+_have_arrow = False
+try:
+import pyarrow
+_have_arrow = True
--- End diff --

Do you mean to automatically enable the Arrow functionality if pyarrow 
installed?  Right now it is enabled manually with a flag `useArrow` in the 
public API `toPandas`.  If enabled and pyarrow is not installed, it will give 
an import error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



  1   2   >