[GitHub] spark pull request: [SPARK-13658][SQL] BooleanSimplification rule ...

2016-03-13 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11647#discussion_r55958696
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
 ---
@@ -598,3 +601,61 @@ abstract class TernaryExpression extends Expression {
 }
   }
 }
+
+/**
+ * Rewrites an expression using rules that are guaranteed preserve the 
result while attempting
+ * to remove cosmetic variations. Deterministic expressions that are 
`equal` after canonicalization
+ * will always return the same answer given the same input (i.e. false 
positives should not be
+ * possible). However, it is possible that two canonical expressions that 
are not equal will in fact
+ * return the same answer given any input (i.e. false negatives are 
possible).
+ *
+ * The following rules are applied:
+ *  - Names and nullability hints for 
[[org.apache.spark.sql.types.DataType]]s are stripped.
+ *  - Commutative and associative operations ([[Add]] and [[Multiply]]) 
have their children ordered
+ *by `hashCode`.
+*   - [[EqualTo]] and [[EqualNullSafe]] are reordered by `hashCode`.
+ *  - Other comparisons ([[GreaterThan]], [[LessThan]]) are reversed by 
`hashCode`.
+ */
+object Canonicalize extends {
--- End diff --

fwiw, if you are keeping this class, i think we should just have it in its 
own file. The Expression.scala file is getting pretty long.


---
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-12893][YARN] Fix history URL redirect e...

2016-03-13 Thread jerryshao
Github user jerryshao commented on the pull request:

https://github.com/apache/spark/pull/10821#issuecomment-196154044
  
Hi @vanzin , thanks a lot for your response. I just checked the branch-1.6, 
looks like the behavior (attempt id) is actually changed, and this change is 
introduced in #9182.

Originally `attemptId` is gotten from `spark.yarn.app.attemptId` which is 
set in `ApplicationMaster`.

And in `ApplicationMaster`, the way to get `attemptId` is 
`appAttemptId.getAttemptId().toString()`, so here the attemptId is "1", "2".

But this behavior is changed in master branch. Here we use the full 
`attemptId` rather than attempt counter 
[here](https://github.com/apache/spark/blob/master/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala#L34).

So this affects not only the file name of event log, also the url of 
history server's each application.

Here if we accept the new way of `attempId`, then this url link should be 
updated to the new one. Oppositely if we treat this new way of `attemptId` as a 
regression, then there's no issue here, all we should change is to loop back 
the original `attemptId`.

What's your opinion? @vanzin .


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/11687#discussion_r55957415
  
--- Diff: python/pyspark/streaming/flume.py ---
@@ -111,13 +111,9 @@ def func(event):
 @staticmethod
 def _get_helper(sc):
 try:
-helperClass = 
sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
-
.loadClass("org.apache.spark.streaming.flume.FlumeUtilsPythonHelper")
-return helperClass.newInstance()
-except Py4JJavaError as e:
-# TODO: use --jar once it also work on driver
-if 'ClassNotFoundException' in str(e.java_exception):
--- End diff --

I made this change because the call now fails with a different set of 
exceptions (such as "attempting to call a package") and wanted to err on the 
side of over-displaying the warning message. Let me try to figure out a 
narrower exception pattern match.


---
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: Branch 1.6

2016-03-13 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-XX] [SQL] fast serialization for collec...

2016-03-13 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11664#discussion_r55956646
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -220,7 +222,61 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
* Runs this query returning the result as an array.
*/
   def executeCollect(): Array[InternalRow] = {
-execute().map(_.copy()).collect()
+// Packing the UnsafeRows into byte array for faster serialization.
+// The byte arrays are in the following format:
+// [size] [bytes of UnsafeRow] [size] [bytes of UnsafeRow] ... [-1]
+val byteArrayRdd = execute().mapPartitionsInternal { iter =>
+  new Iterator[Array[Byte]] {
--- End diff --

i also find this more understandable if you just write it imperatively 
within the map partitions; something like

```scala
execute().mapPartitionsInternal { iter =>
  while (iter.hasNext) {
// write each row to a buffer
  }
  Iterator(buffer)
}
```


---
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-XX] [SQL] fast serialization for collec...

2016-03-13 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11664#discussion_r55956582
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -220,7 +222,61 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
* Runs this query returning the result as an array.
*/
   def executeCollect(): Array[InternalRow] = {
-execute().map(_.copy()).collect()
+// Packing the UnsafeRows into byte array for faster serialization.
+// The byte arrays are in the following format:
+// [size] [bytes of UnsafeRow] [size] [bytes of UnsafeRow] ... [-1]
+val byteArrayRdd = execute().mapPartitionsInternal { iter =>
+  new Iterator[Array[Byte]] {
+private var row: UnsafeRow = _
+override def hasNext: Boolean = row != null || iter.hasNext
+override def next: Array[Byte] = {
--- End diff --

next() rather than next


---
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-XX] [SQL] fast serialization for collec...

2016-03-13 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11664#discussion_r55956568
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -220,7 +222,61 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
* Runs this query returning the result as an array.
*/
   def executeCollect(): Array[InternalRow] = {
-execute().map(_.copy()).collect()
+// Packing the UnsafeRows into byte array for faster serialization.
+// The byte arrays are in the following format:
+// [size] [bytes of UnsafeRow] [size] [bytes of UnsafeRow] ... [-1]
+val byteArrayRdd = execute().mapPartitionsInternal { iter =>
+  new Iterator[Array[Byte]] {
+private var row: UnsafeRow = _
+override def hasNext: Boolean = row != null || iter.hasNext
+override def next: Array[Byte] = {
+  var cap = 1 << 20  // 1 MB
+  if (row != null) {
+// the buffered row could be larger than default buffer size
+cap = Math.max(cap, 4 + row.getSizeInBytes + 4) // reverse 4 
bytes for ending mark (-1).
+  }
+  val buffer = ByteBuffer.allocate(cap)
+  if (row != null) {
+buffer.putInt(row.getSizeInBytes)
+row.writeTo(buffer)
+row = null
+  }
+  while (iter.hasNext) {
+row = iter.next().asInstanceOf[UnsafeRow]
--- End diff --

are we always taking UnsafeRow 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-XX] [SQL] fast serialization for collec...

2016-03-13 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11664#discussion_r55956440
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala ---
@@ -220,7 +222,61 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] 
with Logging with Serializ
* Runs this query returning the result as an array.
*/
   def executeCollect(): Array[InternalRow] = {
-execute().map(_.copy()).collect()
+// Packing the UnsafeRows into byte array for faster serialization.
+// The byte arrays are in the following format:
+// [size] [bytes of UnsafeRow] [size] [bytes of UnsafeRow] ... [-1]
+val byteArrayRdd = execute().mapPartitionsInternal { iter =>
+  new Iterator[Array[Byte]] {
+private var row: UnsafeRow = _
+override def hasNext: Boolean = row != null || iter.hasNext
+override def next: Array[Byte] = {
+  var cap = 1 << 20  // 1 MB
+  if (row != null) {
+// the buffered row could be larger than default buffer size
+cap = Math.max(cap, 4 + row.getSizeInBytes + 4) // reverse 4 
bytes for ending mark (-1).
+  }
+  val buffer = ByteBuffer.allocate(cap)
+  if (row != null) {
+buffer.putInt(row.getSizeInBytes)
+row.writeTo(buffer)
+row = null
+  }
+  while (iter.hasNext) {
+row = iter.next().asInstanceOf[UnsafeRow]
+// Reserve last 4 bytes for ending mark
+if (4 + row.getSizeInBytes + 4 <= buffer.remaining()) {
+  buffer.putInt(row.getSizeInBytes)
+  row.writeTo(buffer)
+  row = null
+} else {
+  buffer.putInt(-1)
+  return buffer.array()
+}
+  }
+  buffer.putInt(-1)
+  // copy the used bytes to make it smaller
+  val bytes = new Array[Byte](buffer.limit())
+  System.arraycopy(buffer.array(), 0, bytes, 0, buffer.limit())
+  bytes
+}
+  }
+}
+// Collect the byte arrays back to driver, then decode them as 
UnsafeRows.
+val nFields = schema.length
+byteArrayRdd.collect().flatMap { bytes =>
--- End diff --

i think this block would be more readable if we just write it imperatively, 
e.g.

```scala

val results = new ArrayBuffer

byteArrayRdd.collect().foreach { bytes =>
  var sizeOfNextRow = bytes.getInt()
  while (sizeOfNextRow >= 0) {
val row = new UnsafeRow(nFields)
row.pointTo(buffer.array(), Platform.BYTE_ARRAY_OFFSET + 
buffer.position(), sizeInBytes)
buffer.position(buffer.position() + sizeOfNextRow)
results += row
  }
}
results.toArray
```


---
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-13843][Streaming]Remove streaming-flume...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11672#issuecomment-196141331
  
**[Test build #53047 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53047/consoleFull)**
 for PR 11672 at commit 
[`97fcb46`](https://github.com/apache/spark/commit/97fcb46152bb8a02e861a7a91443781f4cc3b595).


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/11687#discussion_r55955861
  
--- Diff: python/pyspark/streaming/mqtt.py ---
@@ -38,18 +38,14 @@ def createStream(ssc, brokerUrl, topic,
 :param storageLevel:  RDD storage level.
 :return: A DStream object
 """
-jlevel = ssc._sc._getJavaStorageLevel(storageLevel)
-
 try:
-helperClass = 
ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
-
.loadClass("org.apache.spark.streaming.mqtt.MQTTUtilsPythonHelper")
-helper = helperClass.newInstance()
-jstream = helper.createStream(ssc._jssc, brokerUrl, topic, 
jlevel)
-except Py4JJavaError as e:
-if 'ClassNotFoundException' in str(e.java_exception):
--- End diff --

Same as the above comment


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/11687#discussion_r55955853
  
--- Diff: python/pyspark/streaming/kinesis.py ---
@@ -74,16 +74,13 @@ def createStream(ssc, kinesisAppName, streamName, 
endpointUrl, regionName,
 
 try:
 # Use KinesisUtilsPythonHelper to access Scala's KinesisUtils
-helperClass = 
ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\
-
.loadClass("org.apache.spark.streaming.kinesis.KinesisUtilsPythonHelper")
-helper = helperClass.newInstance()
-jstream = helper.createStream(ssc._jssc, kinesisAppName, 
streamName, endpointUrl,
-  regionName, 
initialPositionInStream, jduration, jlevel,
-  awsAccessKeyId, awsSecretKey)
-except Py4JJavaError as e:
-if 'ClassNotFoundException' in str(e.java_exception):
--- End diff --

Same as the above comment


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/11687#discussion_r55955847
  
--- Diff: python/pyspark/streaming/kafka.py ---
@@ -192,14 +192,9 @@ def funcWithMessageHandler(m):
 @staticmethod
 def _get_helper(sc):
 try:
-# Use KafkaUtilsPythonHelper to access Scala's KafkaUtils (see 
SPARK-6027)
-helperClass = 
sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
-
.loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper")
-return helperClass.newInstance()
-except Py4JJavaError as e:
-# TODO: use --jar once it also work on driver
-if 'ClassNotFoundException' in str(e.java_exception):
--- End diff --

Same as the above comment


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/11687#discussion_r55955831
  
--- Diff: python/pyspark/streaming/flume.py ---
@@ -111,13 +111,9 @@ def func(event):
 @staticmethod
 def _get_helper(sc):
 try:
-helperClass = 
sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
-
.loadClass("org.apache.spark.streaming.flume.FlumeUtilsPythonHelper")
-return helperClass.newInstance()
-except Py4JJavaError as e:
-# TODO: use --jar once it also work on driver
-if 'ClassNotFoundException' in str(e.java_exception):
--- End diff --

Let's still keep this check. For other errors (e.g., the py4j java server 
is down), we should not call `_printErrorMsg` as it's confusing.


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11689#issuecomment-196132481
  
**[Test build #2632 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2632/consoleFull)**
 for PR 11689 at commit 
[`e2d3def`](https://github.com/apache/spark/commit/e2d3def51a91d21ae2037d24ccfde403d78c1040).


---
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-13845][CORE]Using onBlockUpdated to rep...

2016-03-13 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11679#issuecomment-196132369
  
cc @andrewor14 


---
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-4036][MLlib]Add Conditional Random Fiel...

2016-03-13 Thread HuJiayin
Github user HuJiayin commented on the pull request:

https://github.com/apache/spark/pull/9794#issuecomment-196132382
  
@jkbradley I upload the continuous works at 
https://github.com/Intel-bigdata/CRF Thanks very much.


---
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-13840] [SQL] Disable Project Pushdown T...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11682#issuecomment-196131490
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53046/
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-13840] [SQL] Disable Project Pushdown T...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11682#issuecomment-196131489
  
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-13840] [SQL] Disable Project Pushdown T...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11682#issuecomment-196131389
  
**[Test build #53046 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53046/consoleFull)**
 for PR 11682 at commit 
[`e5e00ae`](https://github.com/apache/spark/commit/e5e00ae1d0f9885b05e3a81f8b084e1059151fba).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13823] [CORE] [STREAMING] [SQL] Always ...

2016-03-13 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-13823] [CORE] [STREAMING] [SQL] Always ...

2016-03-13 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11657#issuecomment-196129875
  
Thanks - merging in master.



---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11689#issuecomment-196119705
  
Merged build finished. Test FAILed.


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11689#issuecomment-196119616
  
**[Test build #53045 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53045/consoleFull)**
 for PR 11689 at commit 
[`e2d3def`](https://github.com/apache/spark/commit/e2d3def51a91d21ae2037d24ccfde403d78c1040).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13843][Streaming]Remove streaming-flume...

2016-03-13 Thread harishreedharan
Github user harishreedharan commented on the pull request:

https://github.com/apache/spark/pull/11672#issuecomment-196115717
  
I agree with @ksakellis on this one. It would be great if we can pull Kafka 
out as well. I understand that there are a lot of users who might find it 
difficult, but if you think about it, most people use the plugins via mvn 
anyway (since we don't actually package them in our assembly). I am not sure 
what the policy is if we pull it into a different repo and if we can keep the 
same groupId and artifactId, but that could be an alternative and most likely 
will not break too many users.


---
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-13843][Streaming]Remove streaming-flume...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11672#issuecomment-196111340
  
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-13843][Streaming]Remove streaming-flume...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11672#issuecomment-196111343
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53043/
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-13843][Streaming]Remove streaming-flume...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11672#issuecomment-196111225
  
**[Test build #53043 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53043/consoleFull)**
 for PR 11672 at commit 
[`563614f`](https://github.com/apache/spark/commit/563614f03b6ddb90a1b60c5d569e0045f617ae04).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13658][SQL] BooleanSimplification rule ...

2016-03-13 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/11647#issuecomment-196110966
  
@rxin @marmbrus I've addressed comments. Any other change I should do for 
this? Thanks!


---
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-13840] [SQL] Disable Project Pushdown T...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11682#issuecomment-196110718
  
**[Test build #53046 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53046/consoleFull)**
 for PR 11682 at commit 
[`e5e00ae`](https://github.com/apache/spark/commit/e5e00ae1d0f9885b05e3a81f8b084e1059151fba).


---
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-13834][BUILD] Update sbt and sbt plugin...

2016-03-13 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the pull request:

https://github.com/apache/spark/pull/11669#issuecomment-196110314
  
Thank you, @rxin !


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11687#issuecomment-196106042
  
Merged build finished. Test FAILed.


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11687#issuecomment-196105950
  
**[Test build #53042 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53042/consoleFull)**
 for PR 11687 at commit 
[`15e23f8`](https://github.com/apache/spark/commit/15e23f8ea2c5cfb43af38af5c10d9330c08ea3f0).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13038] [PySpark] Add load/save to pipel...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11683#issuecomment-196104554
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53041/
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-13038] [PySpark] Add load/save to pipel...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11683#issuecomment-196104553
  
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-13038] [PySpark] Add load/save to pipel...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11683#issuecomment-19610
  
**[Test build #53041 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53041/consoleFull)**
 for PR 11683 at commit 
[`4ce0244`](https://github.com/apache/spark/commit/4ce0244c85a166c6a94f508df0bb694978cc95b7).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13834][BUILD] Update sbt and sbt plugin...

2016-03-13 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-13834][BUILD] Update sbt and sbt plugin...

2016-03-13 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11669#issuecomment-196101471
  
Merging in master. Thanks!



---
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: [SQL] fix typo in DataSourceRegister

2016-03-13 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SQL] fix typo in DataSourceRegister

2016-03-13 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11686#issuecomment-196100569
  
Thanks - merging in master & branch-1.6. 

As @srowen said, it'd be great to do more of these in a single batch in the 
future.


---
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-13432][SQL] add the source file name an...

2016-03-13 Thread sarutak
Github user sarutak commented on a diff in the pull request:

https://github.com/apache/spark/pull/11301#discussion_r55948705
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala 
---
@@ -418,6 +419,13 @@ abstract class TreeNode[BaseType <: 
TreeNode[BaseType]] extends Product {
 
   override def toString: String = treeString
 
+  def toOriginString: String =
+if (this.origin.callSite.isDefined && 
!this.isInstanceOf[BoundReference]) {
--- End diff --

I meant `col1` and `col2` are parsed and the comment should be like as 
follows.

```
/* input[1, int] @ selectExpr ... */
...
/* input[2, int] @ selectExpr ...*/
...
```



---
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-13432][SQL] add the source file name an...

2016-03-13 Thread sarutak
Github user sarutak commented on a diff in the pull request:

https://github.com/apache/spark/pull/11301#discussion_r55948509
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala 
---
@@ -57,15 +58,15 @@ object CurrentOrigin {
 
   def reset(): Unit = value.set(Origin())
 
-  def setPosition(line: Int, start: Int): Unit = {
+  def setPosition(callSite: String, line: Int, start: Int): Unit = {
 value.set(
-  value.get.copy(line = Some(line), startPosition = Some(start)))
+  value.get.copy(callSite = Some(callSite), line = Some(line), 
startPosition = Some(start)))
   }
 
   def withOrigin[A](o: Origin)(f: => A): A = {
+val current = get
 set(o)
-val ret = try f finally { reset() }
-reset()
+val ret = try f finally { set(current) }
--- End diff --

I think it should be replaced with the real call site but if it's 
difficult, how about just reset the origin and memorize TODO for the future 
work.


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11689#issuecomment-196089623
  
**[Test build #53045 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53045/consoleFull)**
 for PR 11689 at commit 
[`e2d3def`](https://github.com/apache/spark/commit/e2d3def51a91d21ae2037d24ccfde403d78c1040).


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11689#issuecomment-196089393
  
Merged build finished. Test FAILed.


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11689#issuecomment-196089390
  
**[Test build #53044 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53044/consoleFull)**
 for PR 11689 at commit 
[`2749fe6`](https://github.com/apache/spark/commit/2749fe61b548beb3a7464988dba2f455995382a0).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11689#issuecomment-196089301
  
**[Test build #53044 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53044/consoleFull)**
 for PR 11689 at commit 
[`2749fe6`](https://github.com/apache/spark/commit/2749fe61b548beb3a7464988dba2f455995382a0).


---
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: [MINOR][DOCS] Fix more typos in comments/strin...

2016-03-13 Thread dongjoon-hyun
GitHub user dongjoon-hyun opened a pull request:

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

[MINOR][DOCS] Fix more typos in comments/strings.

## What changes were proposed in this pull request?

This PR fixes 136 typos over 108 files:
* 122 typos in comments
* 11 typos in testcase name
* 3 typos in log messages

## How was this patch tested?

Manual.


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

$ git pull https://github.com/dongjoon-hyun/spark fix_more_typos

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

https://github.com/apache/spark/pull/11689.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #11689


commit 2749fe61b548beb3a7464988dba2f455995382a0
Author: Dongjoon Hyun 
Date:   2016-03-13T23:43:54Z

Fix more typos in comments and strings.




---
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-13578] [core] Modify launch scripts to ...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11591#issuecomment-196088121
  
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-13578] [core] Modify launch scripts to ...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11591#issuecomment-196088123
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53039/
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-13578] [core] Modify launch scripts to ...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11591#issuecomment-196087677
  
**[Test build #53039 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53039/consoleFull)**
 for PR 11591 at commit 
[`cc28749`](https://github.com/apache/spark/commit/cc28749aaa4fdbaf49418ff4cbd4572dad25be07).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13843][Streaming]Remove streaming-flume...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11672#issuecomment-196087049
  
**[Test build #53043 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53043/consoleFull)**
 for PR 11672 at commit 
[`563614f`](https://github.com/apache/spark/commit/563614f03b6ddb90a1b60c5d569e0045f617ae04).


---
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-13843][Streaming]Remove streaming-flume...

2016-03-13 Thread zsxwing
Github user zsxwing commented on the pull request:

https://github.com/apache/spark/pull/11672#issuecomment-196086887
  
retest this please


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11687#issuecomment-196086276
  
**[Test build #53042 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53042/consoleFull)**
 for PR 11687 at commit 
[`15e23f8`](https://github.com/apache/spark/commit/15e23f8ea2c5cfb43af38af5c10d9330c08ea3f0).


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11687#issuecomment-196085547
  
Merged build finished. Test FAILed.


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11687#issuecomment-196085501
  
**[Test build #53038 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53038/consoleFull)**
 for PR 11687 at commit 
[`40d22cc`](https://github.com/apache/spark/commit/40d22cc898a496ed34a48cc939837de2a5da7d11).
 * This patch **fails PySpark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13834][BUILD] Update sbt and sbt plugin...

2016-03-13 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the pull request:

https://github.com/apache/spark/pull/11669#issuecomment-196083702
  
Thank you for your review, @JoshRosen !


---
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-13038] [PySpark] Add load/save to pipel...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11683#issuecomment-196083710
  
**[Test build #53041 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53041/consoleFull)**
 for PR 11683 at commit 
[`4ce0244`](https://github.com/apache/spark/commit/4ce0244c85a166c6a94f508df0bb694978cc95b7).


---
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-13335 Use declarative aggregate for coll...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11688#issuecomment-196083189
  
Merged build finished. Test FAILed.


---
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-13335 Use declarative aggregate for coll...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

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


---
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-13335 Use declarative aggregate for coll...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11688#issuecomment-196083185
  
**[Test build #53040 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53040/consoleFull)**
 for PR 11688 at commit 
[`cb95887`](https://github.com/apache/spark/commit/cb95887e38be02d8534614a07c300ed2bad475f2).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class CollectList(child: Expression) extends DeclarativeAggregate 
`


---
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-13335 Use declarative aggregate for coll...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11688#issuecomment-196083100
  
**[Test build #53040 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53040/consoleFull)**
 for PR 11688 at commit 
[`cb95887`](https://github.com/apache/spark/commit/cb95887e38be02d8534614a07c300ed2bad475f2).


---
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-13335 Use declarative aggregate for coll...

2016-03-13 Thread mccheah
GitHub user mccheah opened a pull request:

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

SPARK-13335 Use declarative aggregate for collect_list

The current implementation of collect_list uses the Hive UDAF, which is 
ill-performant especially considering the need for ImperativeAggregate to 
convert between Catalyst types and Scala types.

In the case of collect_list, we can bypass converting the elements of the 
groups and just combine the encompassing arrays.

I added a new unit test in DataFrameAggregateSuite to exercise this code 
path. I don't have a formal comparison between the Hive UDAF and this, 
unfortunately - I did write an imperative aggregate implementation of 
collect_list and found the declarative variant to be significantly faster. More 
specific numbers should be recorded here when we get the chance.

Some things to think about:
- Code generation for the expressions - I'm not too familiar with writing 
code generation pieces, so it would be good to fill this in.
- I wonder if we can do better for the memory allocation in the 
expressions. Right now every updateList() call calls Array.:+ which isn't the 
most efficient.

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

$ git pull https://github.com/palantir/spark 
feature/spark-13335-collect-list

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

https://github.com/apache/spark/pull/11688.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #11688


commit cb95887e38be02d8534614a07c300ed2bad475f2
Author: mcheah 
Date:   2016-03-13T23:10:22Z

SPARK-13335 Use declarative aggregate for collect_list

The current implementation of collect_list uses the Hive UDAF, which is
ill-performant especially considering the need for ImperativeAggregate
to convert between Catalyst types and Scala types.

In the case of collect_list, we can bypass converting the elements of
the groups and just combine the encompassing arrays.




---
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-13578] [core] Modify launch scripts to ...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11591#issuecomment-196062548
  
**[Test build #53039 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53039/consoleFull)**
 for PR 11591 at commit 
[`cc28749`](https://github.com/apache/spark/commit/cc28749aaa4fdbaf49418ff4cbd4572dad25be07).


---
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-13578] [core] Modify launch scripts to ...

2016-03-13 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/11591#issuecomment-196061824
  
Jenkins, retest this please.


---
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-13834][BUILD] Update sbt and sbt plugin...

2016-03-13 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/11669#issuecomment-196061600
  
This seems fine to me. Arguably we could remove the dependency graph and 
revolver plugins now that they're AutoPlugins, since I imagine that a lot of 
people would prefer to just stick them in their global SBT plugins 
configuration rather than having to configure them on a per-project basis. 
Since other people might disagree with this, though, I think it's fine to just 
upgrade them and leave them here for now.

Therefore, LGTM.


---
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-13825][CORE] Upgrade to Scala 2.11.8

2016-03-13 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/11681#issuecomment-196060987
  
Hey @jaceklaskowski, you should run `./dev/test-dependencies.sh 
--replace-manifest` and commit the result in order to get the dependency tests 
to pass.


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/11687#issuecomment-196060096
  
I suppose we could also backport this change to Spark 1.6.x if there's 
interest, but for now I'm mostly concerned with fixing this in 2.0.0 because I 
believe that the bug that this fixes was causing other issues while trying to 
remove our tests' reliance on the Spark assembly.


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11687#issuecomment-196059153
  
**[Test build #53038 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53038/consoleFull)**
 for PR 11687 at commit 
[`40d22cc`](https://github.com/apache/spark/commit/40d22cc898a496ed34a48cc939837de2a5da7d11).


---
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-13848][SPARK-5185] Update to Py4J 0.9.2...

2016-03-13 Thread JoshRosen
GitHub user JoshRosen opened a pull request:

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

[SPARK-13848][SPARK-5185] Update to Py4J 0.9.2 in order to fix classloading 
issue

This patch upgrades Py4J from 0.9.1 to 0.9.2 in order to include a patch 
which modifies Py4J to use the current thread's ContextClassLoader when 
performing reflection / class loading. This is necessary in order to fix 
[SPARK-5185](https://issues.apache.org/jira/browse/SPARK-5185), a longstanding 
issue affecting the use of `--jars` and `--packages` in PySpark.

In order to demonstrate that the fix works, I removed the workarounds which 
were added as part of 
[SPARK-6027](https://issues.apache.org/jira/browse/SPARK-6027) / #4779 and 
other patches.

Py4J diff: https://github.com/bartdag/py4j/compare/0.9.1...0.9.2

/cc @zsxwing @tdas @davies @brkyvz

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

$ git pull https://github.com/JoshRosen/spark py4j-0.9.2

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

https://github.com/apache/spark/pull/11687.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #11687


commit d21301f69bb6cd6bdc816f0c4ac6d8ea0b47e2ad
Author: Josh Rosen 
Date:   2016-03-13T21:00:42Z

[SPARK-13848] Update to Py4J 0.9.2

commit 40d22cc898a496ed34a48cc939837de2a5da7d11
Author: Josh Rosen 
Date:   2016-03-13T21:40:56Z

[SPARK-5185] Update Py4J tests to demonstate that SPARK-5185 has been fixed.




---
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-13812][SPARKR] Fix SparkR lint-r test e...

2016-03-13 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-13812][SPARKR] Fix SparkR lint-r test e...

2016-03-13 Thread shivaram
Github user shivaram commented on the pull request:

https://github.com/apache/spark/pull/11652#issuecomment-196056344
  
Thanks @sun-rui -- LGTM. Merging 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: [SPARK-13746][Tests]stop using deprecated Sync...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11580#issuecomment-196051048
  
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-13746][Tests]stop using deprecated Sync...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11580#issuecomment-196051051
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53035/
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-13746][Tests]stop using deprecated Sync...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11580#issuecomment-196050740
  
**[Test build #53035 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53035/consoleFull)**
 for PR 11580 at commit 
[`bb8b83a`](https://github.com/apache/spark/commit/bb8b83a9dbc442d48a2acdc3c62bfe6646b61178).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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: [SQL] fix typo in DataSourceRegister

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11686#issuecomment-196046228
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53037/
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: [SQL] fix typo in DataSourceRegister

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11686#issuecomment-196046227
  
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: [SQL] fix typo in DataSourceRegister

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11686#issuecomment-196046142
  
**[Test build #53037 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53037/consoleFull)**
 for PR 11686 at commit 
[`22ec56b`](https://github.com/apache/spark/commit/22ec56bffd4801f9de424c92388a7c68ba7dcdcf).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13834][BUILD] Update sbt and sbt plugin...

2016-03-13 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the pull request:

https://github.com/apache/spark/pull/11669#issuecomment-196043144
  
Hi, @JoshRosen .
If it's possible, could you review this PR, too? This supersedes #11661 .
This is a build PR, so I think we need more careful review.


---
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-12869] Implemented an improved version ...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10839#issuecomment-196022452
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53036/
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-12869] Implemented an improved version ...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10839#issuecomment-196022449
  
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-12869] Implemented an improved version ...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10839#issuecomment-196022345
  
**[Test build #53036 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53036/consoleFull)**
 for PR 10839 at commit 
[`67fd902`](https://github.com/apache/spark/commit/67fd90256f8e2a8310ae297e24b291dab01c3cab).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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: [SQL] fix typo in DataSourceRegister

2016-03-13 Thread srowen
Github user srowen commented on the pull request:

https://github.com/apache/spark/pull/11686#issuecomment-196022000
  
OK. Have you checked to see if there are any other similar typos to correct 
while you're at 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: [SPARK-13432][SQL] add the source file name an...

2016-03-13 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/11301#discussion_r55941719
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala 
---
@@ -418,6 +419,13 @@ abstract class TreeNode[BaseType <: 
TreeNode[BaseType]] extends Product {
 
   override def toString: String = treeString
 
+  def toOriginString: String =
+if (this.origin.callSite.isDefined && 
!this.isInstanceOf[BoundReference]) {
--- End diff --

While I ran the following SQL, the generated code seems to have comments 
properly as follows. Although I will try other SQLs on Monday, I would 
appreciate it if you have already had the case that generates comments 
improperly.


val df = (0 to 4).map(i => (i.toString, i.toInt, i.toFloat)).toDF("s", 
"i", "f")
df.registerTempTable("tempTable")
sqlContext.sql("select i, f from tempTable WHERE i <= 3 and f > 
1.0").show() 


Part of the generated code
java
/* 039 */ while (!shouldStop() && inputadapter_input.hasNext()) {
/* 040 */   InternalRow inputadapter_row = (InternalRow) 
inputadapter_input.next();
/* 041 */   /*** CONSUME: Filter ((_2#1 <= 3) && (cast(_3#2 as double) 
> 1.0)) */
/* 042 */   /* input[1, int] */
/* 043 */   int filter_value1 = inputadapter_row.getInt(1);
/* 044 */   /* input[2, float] */
/* 045 */   float filter_value2 = inputadapter_row.getFloat(2);
/* 046 */   
/* 047 */   /* (input[1, int] <= 3) @ sql at Dataframe1.scala:13 */
/* 048 */   boolean filter_value3 = false;
/* 049 */   filter_value3 = filter_value1 <= 3;
/* 050 */   if (!filter_value3) continue;
/* 051 */   
/* 052 */   /* (cast(input[2, float] as double) > 1.0) @ sql at 
Dataframe1.scala:13 */
/* 053 */   /* cast(input[2, float] as double) @ sql at 
Dataframe1.scala:13 */
/* 054 */   boolean filter_isNull7 = false;
/* 055 */   double filter_value7 = -1.0;
/* 056 */   if (!false) {
/* 057 */ filter_value7 = (double) filter_value2;
/* 058 */   }
/* 059 */   
/* 060 */   boolean filter_value6 = false;
/* 061 */   filter_value6 = 
org.apache.spark.util.Utils.nanSafeCompareDoubles(filter_value7, 1.0D) > 0;
/* 062 */   if (!filter_value6) continue;



---
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: [SQL] fix typo in DataSourceRegister

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11686#issuecomment-196017946
  
**[Test build #53037 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53037/consoleFull)**
 for PR 11686 at commit 
[`22ec56b`](https://github.com/apache/spark/commit/22ec56bffd4801f9de424c92388a7c68ba7dcdcf).


---
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: fix typo in DataSourceRegister

2016-03-13 Thread jackylk
GitHub user jackylk opened a pull request:

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

fix typo in DataSourceRegister

## What changes were proposed in this pull request?
fix typo in DataSourceRegister



## How was this patch tested?

found when going through latest code

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

$ git pull https://github.com/jackylk/spark patch-12

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

https://github.com/apache/spark/pull/11686.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #11686


commit 22ec56bffd4801f9de424c92388a7c68ba7dcdcf
Author: Jacky Li 
Date:   2016-03-13T18:40:11Z

fix typo in DataSourceRegister

found when going through latest code




---
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-11638] [Mesos + Docker Bridge networkin...

2016-03-13 Thread radekg
Github user radekg commented on the pull request:

https://github.com/apache/spark/pull/9608#issuecomment-196016688
  
@tnachen Keeping up with the pace of changes to Spark code was a bit too 
much.


---
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-12869] Implemented an improved version ...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/10839#issuecomment-196016183
  
**[Test build #53036 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53036/consoleFull)**
 for PR 10839 at commit 
[`67fd902`](https://github.com/apache/spark/commit/67fd90256f8e2a8310ae297e24b291dab01c3cab).


---
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-13746][Tests]stop using deprecated Sync...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11580#issuecomment-196016180
  
**[Test build #53035 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53035/consoleFull)**
 for PR 11580 at commit 
[`bb8b83a`](https://github.com/apache/spark/commit/bb8b83a9dbc442d48a2acdc3c62bfe6646b61178).


---
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-13746][Tests]stop using deprecated Sync...

2016-03-13 Thread wilson888888888
Github user wilson8 commented on a diff in the pull request:

https://github.com/apache/spark/pull/11580#discussion_r55941313
  
--- Diff: core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala ---
@@ -578,18 +578,27 @@ class CleanerTester(
   }
 
   private def uncleanedResourcesToString = {
+val s1 = {toBeCleanedRDDIds.synchronized {
--- End diff --

@srowen 
Sorry, I overlooked this.  Will change now.  Thanks!


---
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-11714][Mesos] Make Spark on Mesos honor...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11157#issuecomment-196011925
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53034/
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-11714][Mesos] Make Spark on Mesos honor...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11157#issuecomment-196011924
  
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-11714][Mesos] Make Spark on Mesos honor...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11157#issuecomment-196011806
  
**[Test build #53034 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53034/consoleFull)**
 for PR 11157 at commit 
[`e9f37bc`](https://github.com/apache/spark/commit/e9f37bce106d13d188d8baf71043a30628b0adeb).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-13847][SQL] Defer the variable evaluati...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11685#issuecomment-195989733
  
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-13847][SQL] Defer the variable evaluati...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11685#issuecomment-195989734
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53033/
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-13847][SQL] Defer the variable evaluati...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11685#issuecomment-195989643
  
**[Test build #53033 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53033/consoleFull)**
 for PR 11685 at commit 
[`ff54eb5`](https://github.com/apache/spark/commit/ff54eb51067330cabb7647112e29bc7e7b7c299d).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
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-11714][Mesos] Make Spark on Mesos honor...

2016-03-13 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11157#issuecomment-195979339
  
**[Test build #53034 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53034/consoleFull)**
 for PR 11157 at commit 
[`e9f37bc`](https://github.com/apache/spark/commit/e9f37bce106d13d188d8baf71043a30628b0adeb).


---
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-13801][SQL] DataFrame.col should return...

2016-03-13 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11632#issuecomment-195979011
  
Merged build finished. Test FAILed.


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