[GitHub] spark pull request: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-22 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/8835#discussion_r40132036
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala ---
@@ -342,51 +348,57 @@ case class BatchPythonEvaluation(udf: PythonUDF, 
output: Seq[Attribute], child:
   override def canProcessSafeRows: Boolean = true
 
   protected override def doExecute(): RDD[InternalRow] = {
-val childResults = child.execute().map(_.copy())
+val inputRDD = child.execute().map(_.copy())
+val bufferSize = inputRDD.conf.getInt("spark.buffer.size", 65536)
+val reuseWorker = 
inputRDD.conf.getBoolean("spark.python.worker.reuse", defaultValue = true)
 
-val parent = childResults.mapPartitions { iter =>
+inputRDD.mapPartitions { iter =>
   EvaluatePython.registerPicklers()  // register pickler for Row
+
+  // The queue used to buffer input rows so we can drain it to
+  // combine input with output from Python.
+  val queue = new 
java.util.concurrent.ConcurrentLinkedQueue[InternalRow]()
+
   val pickle = new Pickler
   val currentRow = newMutableProjection(udf.children, child.output)()
   val fields = udf.children.map(_.dataType)
   val schema = new StructType(fields.map(t => new StructField("", t, 
true)).toArray)
-  iter.grouped(100).map { inputRows =>
+
+  // Input iterator to Python: input rows are grouped so we send them 
in batches to Python.
+  // For each row, add it to the queue.
+  val inputIterator = iter.grouped(100).map { inputRows =>
 val toBePickled = inputRows.map { row =>
+  queue.add(row)
   EvaluatePython.toJava(currentRow(row), schema)
 }.toArray
 pickle.dumps(toBePickled)
   }
-}
 
-val pyRDD = new PythonRDD(
-  parent,
-  udf.command,
-  udf.envVars,
-  udf.pythonIncludes,
-  false,
-  udf.pythonExec,
-  udf.pythonVer,
-  udf.broadcastVars,
-  udf.accumulator
-).mapPartitions { iter =>
-  val pickle = new Unpickler
-  iter.flatMap { pickedResult =>
-val unpickledBatch = pickle.loads(pickedResult)
-unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala
-  }
-}.mapPartitions { iter =>
+  val context = TaskContext.get()
+
+  // Output iterator for results from Python.
+  val outputIterator = new PythonRunner(
+udf.command,
+udf.envVars,
+udf.pythonIncludes,
+udf.pythonExec,
+udf.pythonVer,
+udf.broadcastVars,
+udf.accumulator,
+bufferSize,
+reuseWorker
+  ).compute(inputIterator, context.partitionId(), context)
+
+  val unpickle = new Unpickler
--- End diff --

Oh, nevermind: I didn't realize that this was already in a giant 
`mapPartitions` call.


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-22 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/8835#discussion_r40131699
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala ---
@@ -342,51 +348,57 @@ case class BatchPythonEvaluation(udf: PythonUDF, 
output: Seq[Attribute], child:
   override def canProcessSafeRows: Boolean = true
 
   protected override def doExecute(): RDD[InternalRow] = {
-val childResults = child.execute().map(_.copy())
+val inputRDD = child.execute().map(_.copy())
+val bufferSize = inputRDD.conf.getInt("spark.buffer.size", 65536)
+val reuseWorker = 
inputRDD.conf.getBoolean("spark.python.worker.reuse", defaultValue = true)
 
-val parent = childResults.mapPartitions { iter =>
+inputRDD.mapPartitions { iter =>
   EvaluatePython.registerPicklers()  // register pickler for Row
+
+  // The queue used to buffer input rows so we can drain it to
+  // combine input with output from Python.
+  val queue = new 
java.util.concurrent.ConcurrentLinkedQueue[InternalRow]()
+
   val pickle = new Pickler
   val currentRow = newMutableProjection(udf.children, child.output)()
   val fields = udf.children.map(_.dataType)
   val schema = new StructType(fields.map(t => new StructField("", t, 
true)).toArray)
-  iter.grouped(100).map { inputRows =>
+
+  // Input iterator to Python: input rows are grouped so we send them 
in batches to Python.
+  // For each row, add it to the queue.
+  val inputIterator = iter.grouped(100).map { inputRows =>
 val toBePickled = inputRows.map { row =>
+  queue.add(row)
   EvaluatePython.toJava(currentRow(row), schema)
 }.toArray
 pickle.dumps(toBePickled)
   }
-}
 
-val pyRDD = new PythonRDD(
-  parent,
-  udf.command,
-  udf.envVars,
-  udf.pythonIncludes,
-  false,
-  udf.pythonExec,
-  udf.pythonVer,
-  udf.broadcastVars,
-  udf.accumulator
-).mapPartitions { iter =>
-  val pickle = new Unpickler
-  iter.flatMap { pickedResult =>
-val unpickledBatch = pickle.loads(pickedResult)
-unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala
-  }
-}.mapPartitions { iter =>
+  val context = TaskContext.get()
+
+  // Output iterator for results from Python.
+  val outputIterator = new PythonRunner(
+udf.command,
+udf.envVars,
+udf.pythonIncludes,
+udf.pythonExec,
+udf.pythonVer,
+udf.broadcastVars,
+udf.accumulator,
+bufferSize,
+reuseWorker
+  ).compute(inputIterator, context.partitionId(), context)
+
+  val unpickle = new Unpickler
--- End diff --

In the old code, it looks like a new `Unpickler` was constructed for each 
partition. Should we use `outputIterator.flatMapPartitions` and move the 
construction of the unpickler to there in order to be safe / more conservative?


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-22 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/8835#discussion_r40131745
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala ---
@@ -342,51 +348,57 @@ case class BatchPythonEvaluation(udf: PythonUDF, 
output: Seq[Attribute], child:
   override def canProcessSafeRows: Boolean = true
 
   protected override def doExecute(): RDD[InternalRow] = {
-val childResults = child.execute().map(_.copy())
+val inputRDD = child.execute().map(_.copy())
+val bufferSize = inputRDD.conf.getInt("spark.buffer.size", 65536)
+val reuseWorker = 
inputRDD.conf.getBoolean("spark.python.worker.reuse", defaultValue = true)
 
-val parent = childResults.mapPartitions { iter =>
+inputRDD.mapPartitions { iter =>
   EvaluatePython.registerPicklers()  // register pickler for Row
+
+  // The queue used to buffer input rows so we can drain it to
+  // combine input with output from Python.
+  val queue = new 
java.util.concurrent.ConcurrentLinkedQueue[InternalRow]()
+
   val pickle = new Pickler
   val currentRow = newMutableProjection(udf.children, child.output)()
   val fields = udf.children.map(_.dataType)
   val schema = new StructType(fields.map(t => new StructField("", t, 
true)).toArray)
-  iter.grouped(100).map { inputRows =>
+
+  // Input iterator to Python: input rows are grouped so we send them 
in batches to Python.
+  // For each row, add it to the queue.
+  val inputIterator = iter.grouped(100).map { inputRows =>
 val toBePickled = inputRows.map { row =>
+  queue.add(row)
   EvaluatePython.toJava(currentRow(row), schema)
 }.toArray
 pickle.dumps(toBePickled)
   }
-}
 
-val pyRDD = new PythonRDD(
-  parent,
-  udf.command,
-  udf.envVars,
-  udf.pythonIncludes,
-  false,
-  udf.pythonExec,
-  udf.pythonVer,
-  udf.broadcastVars,
-  udf.accumulator
-).mapPartitions { iter =>
-  val pickle = new Unpickler
-  iter.flatMap { pickedResult =>
-val unpickledBatch = pickle.loads(pickedResult)
-unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala
-  }
-}.mapPartitions { iter =>
+  val context = TaskContext.get()
+
+  // Output iterator for results from Python.
+  val outputIterator = new PythonRunner(
+udf.command,
+udf.envVars,
+udf.pythonIncludes,
+udf.pythonExec,
+udf.pythonVer,
+udf.broadcastVars,
+udf.accumulator,
+bufferSize,
+reuseWorker
+  ).compute(inputIterator, context.partitionId(), context)
+
+  val unpickle = new Unpickler
   val row = new GenericMutableRow(1)
-  iter.map { result =>
-row(0) = EvaluatePython.fromJava(result, udf.dataType)
-row: InternalRow
-  }
-}
+  val joined = new JoinedRow
--- End diff --

Same comment here, RE: `joined`: should this be inside of a `mapPartitions` 
call?


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-22 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/8835#discussion_r40133051
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala ---
@@ -329,7 +329,13 @@ case class EvaluatePython(
 /**
  * :: DeveloperApi ::
  * Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at 
a time.
- * The input data is zipped with the result of the udf evaluation.
+ *
+ * Python evaluation works by sending the necessary (projected) input data 
via a socket to an
+ * external Python process, and combine the result from the Python process 
with the original row.
+ *
+ * For each row we send to Python, we also put it in a queue. For each 
output row from Python,
+ * we drain the queue to find the original input row. Note that if the 
Python process is way too
+ * slow, this could lead to the queue growing unbounded and eventually run 
out of memory.
--- End diff --

Could we mitigate this by using a 
[LinkedBlockingDeque](https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/LinkedBlockingDeque.html)
 to have the producer-side block on inserts once the queue grows to a certain 
size?


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-22 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/8835#discussion_r40143619
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala ---
@@ -329,7 +329,13 @@ case class EvaluatePython(
 /**
  * :: DeveloperApi ::
  * Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at 
a time.
- * The input data is zipped with the result of the udf evaluation.
+ *
+ * Python evaluation works by sending the necessary (projected) input data 
via a socket to an
+ * external Python process, and combine the result from the Python process 
with the original row.
+ *
+ * For each row we send to Python, we also put it in a queue. For each 
output row from Python,
+ * we drain the queue to find the original input row. Note that if the 
Python process is way too
+ * slow, this could lead to the queue growing unbounded and eventually run 
out of memory.
--- End diff --

Per discussion offline, the only scenario where the queue can grow really 
large is when the Python buffer size has been configured to be very large and 
the UDF result rows are very small. As a result, I think that this comment 
should be expanded / clarified, but this can take place in a followup 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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-22 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-142422244
  
Based on some offline discussion / debate, we've decided to merge this 
patch into both master and branch-1.5. I'm going to merge this 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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-22 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-20 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-141753867
  
  [Test build #42724 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42724/console)
 for   PR 8835 at commit 
[`5e55bf6`](https://github.com/apache/spark/commit/5e55bf60018e53fc67a153e39f3ae028b4c5fd8f).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `  class WriterThread(`



---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-20 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-141753896
  
Merged build finished. Test PASSed.


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-20 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-141746321
  
  [Test build #42719 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42719/console)
 for   PR 8835 at commit 
[`8d3c495`](https://github.com/apache/spark/commit/8d3c49550a17d549c3a3d545766f41fb432fb424).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `  class WriterThread(`



---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-141746337
  
Merged build finished. Test PASSed.


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-141745236
  
Merged build started.


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/8835#discussion_r39926505
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala ---
@@ -342,51 +348,57 @@ case class BatchPythonEvaluation(udf: PythonUDF, 
output: Seq[Attribute], child:
   override def canProcessSafeRows: Boolean = true
 
   protected override def doExecute(): RDD[InternalRow] = {
-val childResults = child.execute().map(_.copy())
+val inputRDD = child.execute()
--- End diff --

good idea


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-141724308
  
  [Test build #42719 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42719/consoleFull)
 for   PR 8835 at commit 
[`8d3c495`](https://github.com/apache/spark/commit/8d3c49550a17d549c3a3d545766f41fb432fb424).


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-141746330
  
  [Test build #42724 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/42724/consoleFull)
 for   PR 8835 at commit 
[`5e55bf6`](https://github.com/apache/spark/commit/5e55bf60018e53fc67a153e39f3ae028b4c5fd8f).


---
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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/8835#discussion_r39926350
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala ---
@@ -342,51 +348,57 @@ case class BatchPythonEvaluation(udf: PythonUDF, 
output: Seq[Attribute], child:
   override def canProcessSafeRows: Boolean = true
 
   protected override def doExecute(): RDD[InternalRow] = {
-val childResults = child.execute().map(_.copy())
+val inputRDD = child.execute()
--- End diff --

I think we should keep the `copy()` 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: [SPARK-10714][SPARK-8632][SPARK-10685][SQL] Re...

2015-09-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/8835#issuecomment-141745174
  
 Merged build triggered.


---
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