[GitHub] spark pull request: [SPARK-3570] Include time to open files in shu...

2015-02-11 Thread sryza
Github user sryza commented on the pull request:

https://github.com/apache/spark/pull/4550#issuecomment-74031446
  
The immediate goal here is to only track time spent opening the final 
shuffle files - e.g. not to include time spent opening spill files that will 
later be merged, right?

If that's the case, we should track the time in `writePartitionedFile`, not 
`spillToPartitionFiles`.


---
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-5760][SPARK-5761] Fix standalone rest p...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4557#issuecomment-74031314
  
  [Test build #27331 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27331/consoleFull)
 for   PR 4557 at commit 
[`ea48f65`](https://github.com/apache/spark/commit/ea48f65c758d4ec9e80037357ac35ed4f90f6507).
 * This patch merges cleanly.


---
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-5752][SQL] Don't implicitly convert RDD...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4556#issuecomment-74031311
  
  [Test build #27332 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27332/consoleFull)
 for   PR 4556 at commit 
[`ab58d66`](https://github.com/apache/spark/commit/ab58d66c2238b03328a14711f2e86d7950290088).
 * This patch merges cleanly.


---
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-5752][SQL] Don't implicitly convert RDD...

2015-02-11 Thread rxin
GitHub user rxin opened a pull request:

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

[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames

The old implicit would convert RDDs directly to DataFrames, and that added 
too many methods. 

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

$ git pull https://github.com/rxin/spark SPARK-5752

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

https://github.com/apache/spark/pull/4556.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 #4556


commit ab58d66c2238b03328a14711f2e86d7950290088
Author: Reynold Xin 
Date:   2015-02-12T07:55:01Z

[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames




---
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-5760][SPARK-5761] Fix standalone rest p...

2015-02-11 Thread andrewor14
GitHub user andrewor14 opened a pull request:

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

[SPARK-5760][SPARK-5761] Fix standalone rest protocol corner cases + revamp 
tests

The changes are summarized in the commit message. Test or test-related code 
accounts for 90% of the lines changed.

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

$ git pull https://github.com/andrewor14/spark rest-tests

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

https://github.com/apache/spark/pull/4557.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 #4557


commit 00999a85ffa78ab4776b3ef86da1eba475f223a9
Author: Andrew Or 
Date:   2015-02-12T07:43:25Z

Revamp tests + fix a few corner cases

Major changes include:

(1) The server used to always send the JSON to the servlet's
output stream. However, if the response code is not 200,
the client must actually read from the error stream. Now
the server writes to the correct stream depending on the
response code.

(2) If the server throws an internal exception, both the
output stream and the error stream on the client side
is null (not empty). This was not previously accounted
for and could lead to NPEs.

(3) The default error handling servlet did not match the
URL cases correctly, because there used to always be
an empty string in the list. This is now filtered out.




---
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-3570] Include time to open files in shu...

2015-02-11 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/4550#discussion_r24563666
  
--- Diff: 
core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala ---
@@ -352,6 +352,7 @@ private[spark] class ExternalSorter[K, V, C](
 // Create our file writers if we haven't done so yet
 if (partitionWriters == null) {
   curWriteMetrics = new ShuffleWriteMetrics()
+  val openStartTime = System.nanoTime()
--- End diff --

Haha @rxin just chatted me the same thing. FYI I realized I missed some 
file accesses here to add to the shuffle write time; currently testing on a 
cluster to make sure I got them all and will update this patch accordingly.


---
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-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74030756
  
  [Test build #27328 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27328/consoleFull)
 for   PR 4382 at commit 
[`73922ae`](https://github.com/apache/spark/commit/73922ae23f64997e6ee0d30fb74f6d80d38d385e).
 * 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-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74030760
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27328/
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-3570] Include time to open files in shu...

2015-02-11 Thread sryza
Github user sryza commented on a diff in the pull request:

https://github.com/apache/spark/pull/4550#discussion_r24563571
  
--- Diff: 
core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala ---
@@ -352,6 +352,7 @@ private[spark] class ExternalSorter[K, V, C](
 // Create our file writers if we haven't done so yet
 if (partitionWriters == null) {
   curWriteMetrics = new ShuffleWriteMetrics()
+  val openStartTime = System.nanoTime()
--- End diff --

We should be consistent with parens


---
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: Merge pull request #1 from apache/master

2015-02-11 Thread sryza
Github user sryza commented on the pull request:

https://github.com/apache/spark/pull/4553#issuecomment-74030173
  
Hi @hxfeng did you mean to file this?  If not, mind closing 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-5613: Catch the ApplicationNotFoundExcep...

2015-02-11 Thread lianhuiwang
Github user lianhuiwang commented on the pull request:

https://github.com/apache/spark/pull/4392#issuecomment-74030105
  
i think for yarn-cluster mode, we also should catch the 
ApplicationNotFoundException.@pwendell @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-5529][CORE]Add expireDeadHosts in Heart...

2015-02-11 Thread sryza
Github user sryza commented on the pull request:

https://github.com/apache/spark/pull/4363#issuecomment-74029575
  
Thanks for updating this @shenh062326.  I left a few additional comments.  
It would probably be good for @andrewor14 to take a look as well.


---
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-5529][CORE]Add expireDeadHosts in Heart...

2015-02-11 Thread sryza
Github user sryza commented on a diff in the pull request:

https://github.com/apache/spark/pull/4363#discussion_r24563260
  
--- Diff: core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala ---
@@ -17,33 +17,84 @@
 
 package org.apache.spark
 
-import akka.actor.Actor
+import scala.concurrent.duration._
+import scala.collection.mutable
+
+import akka.actor.{Actor, Cancellable}
+
 import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.storage.BlockManagerId
-import org.apache.spark.scheduler.TaskScheduler
+import org.apache.spark.scheduler.{SlaveLost, TaskScheduler}
 import org.apache.spark.util.ActorLogReceive
 
 /**
  * A heartbeat from executors to the driver. This is a shared message used 
by several internal
- * components to convey liveness or execution information for in-progress 
tasks.
+ * components to convey liveness or execution information for in-progress 
tasks. It will also 
+ * expire the hosts that have not heartbeated for more than 
spark.driver.executorTimeoutMs.
  */
 private[spark] case class Heartbeat(
 executorId: String,
 taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics
 blockManagerId: BlockManagerId)
 
+private[spark] case object ExpireDeadHosts 
+
 private[spark] case class HeartbeatResponse(reregisterBlockManager: 
Boolean)
 
 /**
  * Lives in the driver to receive heartbeats from executors..
  */
-private[spark] class HeartbeatReceiver(scheduler: TaskScheduler)
+private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: 
TaskScheduler)
   extends Actor with ActorLogReceive with Logging {
 
+  val executorLastSeen = new mutable.HashMap[String, Long]
+  
+  val executorTimeout = sc.conf.getLong("spark.driver.executorTimeoutMs", 
+sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120 * 
1000))
+  
+  val checkTimeoutInterval = 
sc.conf.getLong("spark.driver.executorTimeoutIntervalMs",
+sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 6))
+  
+  var timeoutCheckingTask: Cancellable = null
+  
+  override def preStart(): Unit = {
+import context.dispatcher
+timeoutCheckingTask = context.system.scheduler.schedule(0.seconds,
+  checkTimeoutInterval.milliseconds, self, ExpireDeadHosts)
+super.preStart()
+  }
+  
   override def receiveWithLogging = {
 case Heartbeat(executorId, taskMetrics, blockManagerId) =>
   val response = HeartbeatResponse(
 !scheduler.executorHeartbeatReceived(executorId, taskMetrics, 
blockManagerId))
+  executorLastSeen(executorId) = System.currentTimeMillis()
   sender ! response
+case ExpireDeadHosts =>
+  expireDeadHosts()
+  }
+
+  private def expireDeadHosts(): Unit = {
+logTrace("Checking for hosts with no recent heartbeats in 
HeartbeatReceiver.")
+val now = System.currentTimeMillis()
+val minSeenTime = now - executorTimeout
+for ((executorId, lastSeenMs) <- executorLastSeen) {
+  if (lastSeenMs < minSeenTime) {
+logWarning(s"Removing executor $executorId with no recent 
heartbeats: " +
+  s"${now - lastSeenMs} ms exceeds timeout $executorTimeout ms")
+scheduler.executorLost(executorId, SlaveLost())
+if(sc.supportKillExecutor()) {
+  sc.killExecutor(executorId)
+}
+executorLastSeen.remove(executorId)
--- End diff --

What happens if a heartbeat from the executor gets delivered after we kill 
/ remove 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-5529][CORE]Add expireDeadHosts in Heart...

2015-02-11 Thread sryza
Github user sryza commented on a diff in the pull request:

https://github.com/apache/spark/pull/4363#discussion_r24563184
  
--- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
@@ -1034,6 +1034,14 @@ class SparkContext(config: SparkConf) extends 
Logging with ExecutorAllocationCli
 logInfo("Added file " + path + " at " + key + " with timestamp " + 
addedFiles(key))
 postEnvironmentUpdate()
   }
+  
+  def supportKillExecutor(): Boolean = {
--- End diff --

I would call this `supportKillExecutor` and it should be made 
`private[spark]`.

Also can be shortened to just `private[spark] def supportsKillExecutor = 
master.contains("yarn") || dynamicAllocationTesting`.


---
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-5529][CORE]Add expireDeadHosts in Heart...

2015-02-11 Thread sryza
Github user sryza commented on a diff in the pull request:

https://github.com/apache/spark/pull/4363#discussion_r24563093
  
--- Diff: core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala ---
@@ -17,33 +17,84 @@
 
 package org.apache.spark
 
-import akka.actor.Actor
+import scala.concurrent.duration._
+import scala.collection.mutable
+
+import akka.actor.{Actor, Cancellable}
+
 import org.apache.spark.executor.TaskMetrics
 import org.apache.spark.storage.BlockManagerId
-import org.apache.spark.scheduler.TaskScheduler
+import org.apache.spark.scheduler.{SlaveLost, TaskScheduler}
 import org.apache.spark.util.ActorLogReceive
 
 /**
  * A heartbeat from executors to the driver. This is a shared message used 
by several internal
- * components to convey liveness or execution information for in-progress 
tasks.
+ * components to convey liveness or execution information for in-progress 
tasks. It will also 
+ * expire the hosts that have not heartbeated for more than 
spark.driver.executorTimeoutMs.
  */
 private[spark] case class Heartbeat(
 executorId: String,
 taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics
 blockManagerId: BlockManagerId)
 
+private[spark] case object ExpireDeadHosts 
+
 private[spark] case class HeartbeatResponse(reregisterBlockManager: 
Boolean)
 
 /**
  * Lives in the driver to receive heartbeats from executors..
  */
-private[spark] class HeartbeatReceiver(scheduler: TaskScheduler)
+private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: 
TaskScheduler)
   extends Actor with ActorLogReceive with Logging {
 
+  val executorLastSeen = new mutable.HashMap[String, Long]
+  
+  val executorTimeout = sc.conf.getLong("spark.driver.executorTimeoutMs", 
+sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120 * 
1000))
+  
+  val checkTimeoutInterval = 
sc.conf.getLong("spark.driver.executorTimeoutIntervalMs",
+sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 6))
+  
+  var timeoutCheckingTask: Cancellable = null
+  
+  override def preStart(): Unit = {
+import context.dispatcher
+timeoutCheckingTask = context.system.scheduler.schedule(0.seconds,
+  checkTimeoutInterval.milliseconds, self, ExpireDeadHosts)
+super.preStart()
+  }
+  
   override def receiveWithLogging = {
 case Heartbeat(executorId, taskMetrics, blockManagerId) =>
   val response = HeartbeatResponse(
 !scheduler.executorHeartbeatReceived(executorId, taskMetrics, 
blockManagerId))
+  executorLastSeen(executorId) = System.currentTimeMillis()
   sender ! response
+case ExpireDeadHosts =>
+  expireDeadHosts()
+  }
+
+  private def expireDeadHosts(): Unit = {
+logTrace("Checking for hosts with no recent heartbeats in 
HeartbeatReceiver.")
+val now = System.currentTimeMillis()
+val minSeenTime = now - executorTimeout
+for ((executorId, lastSeenMs) <- executorLastSeen) {
+  if (lastSeenMs < minSeenTime) {
+logWarning(s"Removing executor $executorId with no recent 
heartbeats: " +
+  s"${now - lastSeenMs} ms exceeds timeout $executorTimeout ms")
+scheduler.executorLost(executorId, SlaveLost())
+if(sc.supportKillExecutor()) {
--- End diff --

Nit: need space after `if`


---
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-575][MLLIB] replace SQL JSON usage in m...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4555#issuecomment-74028632
  
  [Test build #27330 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27330/consoleFull)
 for   PR 4555 at commit 
[`b0415e8`](https://github.com/apache/spark/commit/b0415e84e44317b8397e9ffb9534c092582cb9d4).
 * This patch merges cleanly.


---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74028451
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27325/
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74028439
  
  [Test build #27325 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27325/consoleFull)
 for   PR 4547 at commit 
[`acbb281`](https://github.com/apache/spark/commit/acbb281567ab110fee5b47bd97689996b2c94db0).
 * 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-575][MLLIB] replace SQL JSON usage in m...

2015-02-11 Thread mengxr
GitHub user mengxr opened a pull request:

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

[SPARK-575][MLLIB] replace SQL JSON usage in model import/export by json4s

This PR detaches MLlib model import/export code from SQL's JSON support. 
@yhuai

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

$ git pull https://github.com/mengxr/spark SPARK-5757

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

https://github.com/apache/spark/pull/4555.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 #4555


commit b0415e84e44317b8397e9ffb9534c092582cb9d4
Author: Xiangrui Meng 
Date:   2015-02-12T07:21:16Z

replace SQL JSON usage by json4s




---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74027990
  
  [Test build #27324 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27324/consoleFull)
 for   PR 4547 at commit 
[`7793dcb`](https://github.com/apache/spark/commit/7793dcbb5f9f1ec66b539eb4bb5b7ad9cac1602a).
 * 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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74027997
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27324/
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-5759][Yarn]ExecutorRunnable should catc...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4554#issuecomment-74027809
  
  [Test build #27329 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27329/consoleFull)
 for   PR 4554 at commit 
[`c02140f`](https://github.com/apache/spark/commit/c02140f67c8e7d36675b3ec20f27d25e456900be).
 * This patch merges cleanly.


---
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-5759][Yarn]ExecutorRunnable should catc...

2015-02-11 Thread lianhuiwang
GitHub user lianhuiwang opened a pull request:

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

[SPARK-5759][Yarn]ExecutorRunnable should catch YarnException while 
NMClient start contain...

some time since some of reasons, it lead to some exception while NMClient 
start container.example:we do not config spark_shuffle on some machines, so it 
will throw a exception:
java.lang.Error: 
org.apache.hadoop.yarn.exceptions.InvalidAuxServiceException: The 
auxService:spark_shuffle does not exist.
because YarnAllocator use ThreadPoolExecutor to start Container, so we can 
not find which container or hostname throw exception. I think we should catch 
YarnException in ExecutorRunnable when start container. if there are some 
exceptions, we can know the container id or hostname of failed container.

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

$ git pull https://github.com/lianhuiwang/spark SPARK-5759

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

https://github.com/apache/spark/pull/4554.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 #4554


commit c02140f67c8e7d36675b3ec20f27d25e456900be
Author: lianhuiwang 
Date:   2015-02-12T07:11:19Z

ExecutorRunnable should catch YarnException while NMClient start container




---
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-5756][SQL] Analyzer should not throw sc...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4552#issuecomment-74027544
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27327/
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-5756][SQL] Analyzer should not throw sc...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4552#issuecomment-74027537
  
  [Test build #27327 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27327/consoleFull)
 for   PR 4552 at commit 
[`58690c0`](https://github.com/apache/spark/commit/58690c01b0c6238c60ac8640d6cbdd2a095490ab).
 * 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-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74026567
  
  [Test build #27328 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27328/consoleFull)
 for   PR 4382 at commit 
[`73922ae`](https://github.com/apache/spark/commit/73922ae23f64997e6ee0d30fb74f6d80d38d385e).
 * This patch merges cleanly.


---
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-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74026303
  
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-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74026286
  
Seems failure due to the irrelevant 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: Merge pull request #1 from apache/master

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4553#issuecomment-74025053
  
Can one of the admins verify this patch?


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

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



[GitHub] spark pull request: [SPARK-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74024949
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27326/
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-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74024944
  
  [Test build #27326 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27326/consoleFull)
 for   PR 4382 at commit 
[`73922ae`](https://github.com/apache/spark/commit/73922ae23f64997e6ee0d30fb74f6d80d38d385e).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `trait DataFrame extends RDDApi[Row] with Serializable `



---
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: Merge pull request #1 from apache/master

2015-02-11 Thread hxfeng
GitHub user hxfeng opened a pull request:

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

Merge pull request #1 from apache/master

update

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

$ git pull https://github.com/hxfeng/spark master

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

https://github.com/apache/spark/pull/4553.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 #4553


commit b3ee640ffa59ed14fdeba61d5bf53b9b8e6cc520
Author: hxfeng <980548...@qq.com>
Date:   2014-12-28T03:51:35Z

Merge pull request #1 from apache/master

update




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

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



[GitHub] spark pull request: [SPARK-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74024214
  
I agree with @liancheng, too, both code and description are updated.


---
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-5756][SQL] Analyzer should not throw sc...

2015-02-11 Thread scwf
Github user scwf commented on the pull request:

https://github.com/apache/spark/pull/4552#issuecomment-74023858
  
maybe we should catch the exception in typeSuffix to fix 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-5529][CORE]Add expireDeadHosts in Heart...

2015-02-11 Thread shenh062326
Github user shenh062326 commented on the pull request:

https://github.com/apache/spark/pull/4363#issuecomment-74023636
  
Hi @sryza, I think this pull request is OK now, can you merge it into 
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: [SPARK-5495][UI] Add app kill function in mast...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4288#issuecomment-74023153
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27323/
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-5495][UI] Add app kill function in mast...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4288#issuecomment-74023146
  
  [Test build #27323 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27323/consoleFull)
 for   PR 4288 at commit 
[`2495c49`](https://github.com/apache/spark/commit/2495c49406baa4a1516bdbe91734520703a1cff4).
 * 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-5755] [SQL] remove unnecessary Add

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4551#issuecomment-74022508
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27322/
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-5755] [SQL] remove unnecessary Add

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4551#issuecomment-74022499
  
  [Test build #27322 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27322/consoleFull)
 for   PR 4551 at commit 
[`0821ae4`](https://github.com/apache/spark/commit/0821ae49413c80c7808caa5d48a12df8729d2e43).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `trait DataFrame extends RDDApi[Row] with Serializable `



---
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] Use LongType as the default type for int...

2015-02-11 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/4544#issuecomment-74022077
  
@yhuai I created a JIRA to use json4s directly in MLlib: 
https://issues.apache.org/jira/browse/SPARK-5757. Could you make a JIRA for 
this PR and link them?


---
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-5649][SQL] added a rule to check dataty...

2015-02-11 Thread scwf
Github user scwf commented on a diff in the pull request:

https://github.com/apache/spark/pull/4425#discussion_r24560220
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -69,6 +69,7 @@ class Analyzer(catalog: Catalog,
   typeCoercionRules ++
   extendedRules : _*),
 Batch("Check Analysis", Once,
+  CheckCast ::
--- End diff --

scala.NotImplementedError should be a bug, filed PR #4552 for 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-5756][SQL] Analyzer should not throw sc...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4552#issuecomment-74021800
  
  [Test build #27327 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27327/consoleFull)
 for   PR 4552 at commit 
[`58690c0`](https://github.com/apache/spark/commit/58690c01b0c6238c60ac8640d6cbdd2a095490ab).
 * This patch merges cleanly.


---
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-5756][SQL] Analyzer should not throw sc...

2015-02-11 Thread scwf
GitHub user scwf opened a pull request:

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

[SPARK-5756][SQL] Analyzer should not throw scala.NotImplementedError for 
legitimate sql

```SELECT CAST(x AS STRING) FROM src```  comes a NotImplementedError:
 
 CliDriver: scala.NotImplementedError: an implementation is missing
at scala.Predef$.$qmark$qmark$qmark(Predef.scala:252)
at 
org.apache.spark.sql.catalyst.expressions.PrettyAttribute.dataType(namedExpressions.scala:221)
at 
org.apache.spark.sql.catalyst.expressions.Cast.resolved$lzycompute(Cast.scala:30)
at 
org.apache.spark.sql.catalyst.expressions.Cast.resolved(Cast.scala:30)
at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:68)
at 
org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:68)
at 
scala.collection.LinearSeqOptimized$class.exists(LinearSeqOptimized.scala:80)
at scala.collection.immutable.List.exists(List.scala:84)
at 
org.apache.spark.sql.catalyst.expressions.Expression.childrenResolved(Expression.scala:68)
at 
org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:56)
at 
org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:56)
at 
org.apache.spark.sql.catalyst.expressions.NamedExpression.typeSuffix(namedExpressions.scala:62)
at 
org.apache.spark.sql.catalyst.expressions.Alias.toString(namedExpressions.scala:124)
at 
org.apache.spark.sql.catalyst.expressions.Expression.prettyString(Expression.scala:78)
at 
org.apache.spark.sql.catalyst.analysis.Analyzer$CheckResolution$$anonfun$1$$anonfun$7.apply(Analyzer.scala:83)
at 
org.apache.spark.sql.catalyst.analysis.Analyzer$CheckResolution$$anonfun$1$$anonfun$7.apply(Analyzer.scala:83)
at scala.collection.immutable.Stream.map(Stream.scala:376)
at 
org.apache.spark.sql.catalyst.analysis.Analyzer$CheckResolution$$anonfun$1.applyOrElse(Analyzer.scala:83)
at 
org.apache.spark.sql.catalyst.analysis.Analyzer$CheckResolution$$anonfun$1.applyOrElse(Analyzer.scala:81)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:204)
at 
org.apache.spark.sql.catalyst.analysis.Analyzer$CheckResolution$.apply(Analyzer.scala:81)
at 
org.apache.spark.sql.catalyst.analysis.Analyzer$CheckResolution$.apply(Analyzer.scala:79)


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

$ git pull https://github.com/scwf/spark implemented-error

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

https://github.com/apache/spark/pull/4552.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 #4552


commit 58690c01b0c6238c60ac8640d6cbdd2a095490ab
Author: wangfei 
Date:   2015-02-12T05:50:24Z

fix implemented-error




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

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



[GitHub] spark pull request: [SPARK-2087] [SQL] Multiple thriftserver sessi...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4382#issuecomment-74021473
  
  [Test build #27326 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27326/consoleFull)
 for   PR 4382 at commit 
[`73922ae`](https://github.com/apache/spark/commit/73922ae23f64997e6ee0d30fb74f6d80d38d385e).
 * This patch merges cleanly.


---
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] Use LongType as the default type for int...

2015-02-11 Thread mengxr
Github user mengxr commented on the pull request:

https://github.com/apache/spark/pull/4544#issuecomment-74020725
  
Yes, we use `row.getInt()` in MLlib. I think we should use json4s directly 
in MLlib's export/import.


---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74020520
  
  [Test build #27325 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27325/consoleFull)
 for   PR 4547 at commit 
[`acbb281`](https://github.com/apache/spark/commit/acbb281567ab110fee5b47bd97689996b2c94db0).
 * This patch merges cleanly.


---
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-5522] Accelerate the Histroty Server st...

2015-02-11 Thread marsishandsome
Github user marsishandsome commented on the pull request:

https://github.com/apache/spark/pull/4525#issuecomment-74020260
  
@vanzin Thanks for your advice. I will improve the implementation.


---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74020191
  
  [Test build #27324 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27324/consoleFull)
 for   PR 4547 at commit 
[`7793dcb`](https://github.com/apache/spark/commit/7793dcbb5f9f1ec66b539eb4bb5b7ad9cac1602a).
 * This patch merges cleanly.


---
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-5722][SQL] fix for infer long type in p...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4521#issuecomment-74019267
  
  [Test build #599 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/599/consoleFull)
 for   PR 4521 at commit 
[`41f363a`](https://github.com/apache/spark/commit/41f363a3e4270f9d855facb08b0ee51700c78a2a).
 * This patch **fails Python 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: [SPARK-5722][SQL] fix for infer long type in p...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4521#issuecomment-74019193
  
  [Test build #599 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/599/consoleFull)
 for   PR 4521 at commit 
[`41f363a`](https://github.com/apache/spark/commit/41f363a3e4270f9d855facb08b0ee51700c78a2a).
 * This patch merges cleanly.


---
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-5722][SQL] fix for infer long type in p...

2015-02-11 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/4521#issuecomment-74019151
  
@dondrake You could run the python sql tests locally, and see many tests 
are failing.

The size of int/long in Python is determined by size of machine word, see 
sys.maxint. In 32-bit machine, they should be Integer and BigInteger. In 64-bit 
machine, they should be Long and BigInteger.

From user's point view, I think Integer is enough for most of cases, the 
original patch could help in some cases (nanoseconds and uuid). In the cases 
that the inferred type is not correct, user could specify the correct type 
manually. I think this is the behavior without surprise to 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-5495][UI] Add app kill function in mast...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4288#issuecomment-74017808
  
  [Test build #27323 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27323/consoleFull)
 for   PR 4288 at commit 
[`2495c49`](https://github.com/apache/spark/commit/2495c49406baa4a1516bdbe91734520703a1cff4).
 * This patch merges cleanly.


---
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-5645] Added local read bytes/time to ta...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74017678
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27321/
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-5645] Added local read bytes/time to ta...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74017676
  
  [Test build #27321 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27321/consoleFull)
 for   PR 4510 at commit 
[`4a0182c`](https://github.com/apache/spark/commit/4a0182c294d8d96eba87992250cc61cdbb0d997d).
 * 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-5755] [SQL] remove unnecessary Add

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4551#issuecomment-74017185
  
  [Test build #27322 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27322/consoleFull)
 for   PR 4551 at commit 
[`0821ae4`](https://github.com/apache/spark/commit/0821ae49413c80c7808caa5d48a12df8729d2e43).
 * This patch merges cleanly.


---
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-5755] [SQL] remove unnecessary Add

2015-02-11 Thread adrian-wang
GitHub user adrian-wang opened a pull request:

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

[SPARK-5755] [SQL] remove unnecessary Add

explain extended select +key from src;
before:
== Parsed Logical Plan ==
'Project [(0 + 'key) AS _c0#8]
 'UnresolvedRelation [src], None

== Analyzed Logical Plan ==
Project [(0 + key#10) AS _c0#8]
 MetastoreRelation test, src, None

== Optimized Logical Plan ==
Project [(0 + key#10) AS _c0#8]
 MetastoreRelation test, src, None

== Physical Plan ==
Project [(0 + key#10) AS _c0#8]
 HiveTableScan [key#10], (MetastoreRelation test, src, None), None


after this patch:
== Parsed Logical Plan ==
'Project ['key]
 'UnresolvedRelation [src], None

== Analyzed Logical Plan ==
Project [key#10]
 MetastoreRelation test, src, None

== Optimized Logical Plan ==
Project [key#10]
 MetastoreRelation test, src, None

== Physical Plan ==
HiveTableScan [key#10], (MetastoreRelation test, src, None), None

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

$ git pull https://github.com/adrian-wang/spark positive

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

https://github.com/apache/spark/pull/4551.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 #4551


commit 0821ae49413c80c7808caa5d48a12df8729d2e43
Author: Daoyuan Wang 
Date:   2015-02-12T04:39:13Z

remove unnecessary Add




---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread marmbrus
Github user marmbrus commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74016049
  
You could just add and remove a table to make sure the list reflects this
change.
On Feb 11, 2015 8:29 PM, "Yin Huai"  wrote:

> Actually, I cannot use TestSQLContext and TestHive in my test (they have
> lots of tables registered by other tests). Will find another way.
>
> —
> Reply to this email directly or view it on GitHub
> .
>



---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread yhuai
Github user yhuai commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74015963
  
Actually, I cannot use `TestSQLContext` and `TestHive` in my test (they 
have lots of tables registered by other tests). Will find another way.


---
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-3570] Include time to open files in shu...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4550#issuecomment-74015858
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27317/
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-3570] Include time to open files in shu...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4550#issuecomment-74015854
  
  [Test build #27317 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27317/consoleFull)
 for   PR 4550 at commit 
[`2423555`](https://github.com/apache/spark/commit/24235553adc748ccedf3998b0c174711b214e970).
 * 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-5573][SQL] Add explode to dataframes

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4546#issuecomment-74015642
  
  [Test build #27318 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27318/consoleFull)
 for   PR 4546 at commit 
[`a8d496c`](https://github.com/apache/spark/commit/a8d496cb352c0b9423eeebeee82df69a5c97d2ea).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class UserDefinedGenerator(`
  * `trait DataFrame extends RDDApi[Row] with Serializable `
  * `   *   case class Book(title: String, words: String)`
  * `   *   case class Word(word: String)`



---
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-5573][SQL] Add explode to dataframes

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4546#issuecomment-74015647
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27318/
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-5649][SQL] added a rule to check dataty...

2015-02-11 Thread scwf
Github user scwf commented on a diff in the pull request:

https://github.com/apache/spark/pull/4425#discussion_r24557748
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -69,6 +69,7 @@ class Analyzer(catalog: Catalog,
   typeCoercionRules ++
   extendedRules : _*),
 Batch("Check Analysis", Once,
+  CheckCast ::
--- End diff --

If we check resolution first, 
1 check cast will be invalid since legitimate cast expression must be not 
resolved.
2 ```select cast(key as binary) from src``` will give a 
```scala.NotImplementedError```

how about not change the order but catch exception of child.dataType to 
make it pass this rule?


---
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-5680][SQL] Sum function on all null val...

2015-02-11 Thread adrian-wang
Github user adrian-wang commented on the pull request:

https://github.com/apache/spark/pull/4466#issuecomment-74014984
  
That makes sense, 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-2555] Support configuration spark.sched...

2015-02-11 Thread li-zhihui
Github user li-zhihui commented on the pull request:

https://github.com/apache/spark/pull/1462#issuecomment-74014949
  
@pwendell Do we need the feature in mesos mode? I am pleasure to update 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-3299][SQL] add to SQLContext API to sho...

2015-02-11 Thread bbejeck
Github user bbejeck commented on the pull request:

https://github.com/apache/spark/pull/3872#issuecomment-74014827
  
@yhuai I think it's a good idea, nice 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: [SPARK-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/4547#discussion_r24557118
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala ---
@@ -734,6 +734,24 @@ class SQLContext(@transient val sparkContext: 
SparkContext)
   def table(tableName: String): DataFrame =
 DataFrame(this, catalog.lookupRelation(Seq(tableName)))
 
+  /**
+   * Returns a [[DataFrame]] containing names of existing tables in the 
current database.
+   * The returned DataFrame has two columns, tableName and isTemporary (a 
column with BooleanType
+   * indicating if a table is a temporary one or not).
+   */
+  def tables(databaseName: String): DataFrame = {
--- End diff --

Really, `showTables` or `listTables` both sounds like a `Unit` function 
that prints to the screen (like the `show` function we already have on 
`DataFrame`).


---
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-5682] Reuse hadoop encrypted shuffle al...

2015-02-11 Thread kellyzly
Github user kellyzly commented on the pull request:

https://github.com/apache/spark/pull/4491#issuecomment-74014097
  
@JoshRosen:  I have updated the patch, added the profile of hadoop 2.4 I 
deleted before and add a profile of hadoop2.6.


---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread marmbrus
Github user marmbrus commented on a diff in the pull request:

https://github.com/apache/spark/pull/4547#discussion_r24557066
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
 ---
@@ -34,6 +34,12 @@ trait Catalog {
   tableIdentifier: Seq[String],
   alias: Option[String] = None): LogicalPlan
 
+  /**
+   * Returns names and flags indicating if a table is temporary or not of 
all tables in the
+   * database identified by `databaseIdentifier`.
+   */
+  def getTables(databaseIdentifier: Seq[String]): Seq[(String, Boolean)]
--- End diff --

I'd make this a single string.  `tableIdentifier` is more an internal 
concept.  We can always use a parser here.


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

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



[GitHub] spark pull request: Fix make-distribution.sh by adding quotes to $...

2015-02-11 Thread nchammas
Github user nchammas commented on the pull request:

https://github.com/apache/spark/pull/4540#issuecomment-74012417
  
FWIW, this patch LGTM as-is.


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

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



[GitHub] spark pull request: [SPARK-5645] Added local read bytes/time to ta...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74012381
  
  [Test build #27321 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27321/consoleFull)
 for   PR 4510 at commit 
[`4a0182c`](https://github.com/apache/spark/commit/4a0182c294d8d96eba87992250cc61cdbb0d997d).
 * This patch merges cleanly.


---
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-5645] Added local read bytes/time to ta...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74012184
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27320/
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-5645] Added local read bytes/time to ta...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74012183
  
  [Test build #27320 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27320/consoleFull)
 for   PR 4510 at commit 
[`5f5da1b`](https://github.com/apache/spark/commit/5f5da1b33e0ddffe9d93e3514d3874200a1cbd61).
 * This patch **fails to build**.
 * 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-5645] Added local read bytes/time to ta...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74012021
  
  [Test build #27320 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27320/consoleFull)
 for   PR 4510 at commit 
[`5f5da1b`](https://github.com/apache/spark/commit/5f5da1b33e0ddffe9d93e3514d3874200a1cbd61).
 * This patch merges cleanly.


---
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-5645] Added local read bytes/time to ta...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74011837
  
  [Test build #27319 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27319/consoleFull)
 for   PR 4510 at commit 
[`5da04cf`](https://github.com/apache/spark/commit/5da04cf2dee630416218e5bf18adc8083ef01607).
 * This patch **fails to build**.
 * 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-5645] Added local read bytes/time to ta...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74011838
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27319/
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: [IDE] cannot import src/main/scala & src/test/...

2015-02-11 Thread ligangty
Github user ligangty commented on the pull request:

https://github.com/apache/spark/pull/4531#issuecomment-74011785
  
I'm not sure why it has this behavior, but the eclipse surely act as that. 
I've tested in  Eclipse 3.8 and 4.3 with scala-plugin and scala-maven-plugin in 
both linux and windows platform, and got the same results. And after add this 
add-source goal, the src/main/scala will be imported as source folder which is 
reasonable.


---
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-5645] Added local read bytes/time to ta...

2015-02-11 Thread kayousterhout
Github user kayousterhout commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74011746
  
Thanks for taking a look at this @ksakellis! I've fixed the issues you 
pointed out.


---
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-5645] Added local read bytes/time to ta...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4510#issuecomment-74011712
  
  [Test build #27319 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27319/consoleFull)
 for   PR 4510 at commit 
[`5da04cf`](https://github.com/apache/spark/commit/5da04cf2dee630416218e5bf18adc8083ef01607).
 * This patch merges cleanly.


---
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-5680][SQL] Sum function on all null val...

2015-02-11 Thread gvramana
Github user gvramana commented on the pull request:

https://github.com/apache/spark/pull/4466#issuecomment-74011625
  
Case classes cannot be derived. Making it non-case class will disturb the 
uniformity. so I left them like that.


---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74011428
  
  [Test build #27316 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27316/consoleFull)
 for   PR 4547 at commit 
[`aba2e88`](https://github.com/apache/spark/commit/aba2e889dd67957ad2f884c050ade3c41fe2487e).
 * 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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4547#issuecomment-74011435
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27316/
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-2555] Support configuration spark.sched...

2015-02-11 Thread nchammas
Github user nchammas commented on the pull request:

https://github.com/apache/spark/pull/1462#issuecomment-74011208
  
This PR has gone stale. Do we want to update 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-1192][RESUBMIT]Missing document of some...

2015-02-11 Thread nchammas
Github user nchammas commented on the pull request:

https://github.com/apache/spark/pull/2312#issuecomment-74011120
  
@CodingCat This PR has gone stale. Do you want to update 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-3830][MLlib] Implement genetic algorith...

2015-02-11 Thread nchammas
Github user nchammas commented on the pull request:

https://github.com/apache/spark/pull/2731#issuecomment-74011021
  
@mengxr Did you ever see 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-5681][Streaming] Add tracker status and...

2015-02-11 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/4467#issuecomment-74010907
  
Let's analyze it clearly. The following is a simplified status 
transformation of the problem:

 tracker   receivers
t = 1  started   registered:{A, B}; starting, not registered: {C}
t = 2  stoppinggot stop msg:{A, B}; starting, not registered: {C}
t = 3  stoppingstopped:{A, B}; registered: {C}

The above causes potential data loss. We want to avoid that. I agree.

If we implement option 1, now the status transformation:

 tracker   receivers
t = 1  started   registered:{A, B}; starting, not registered: {C}
t = 2  stoppinggot stop msg:{A, B}; starting, not registered: {C}

*we are going to wait for receivers that are started but not registered yet.
*suppose we wait a fixed time period n.
*however, we can't guarantee when the receiver C will be registered.
*so, after waiting time n, the system status can be:

t = n+2  stoppingstopped:{A, B}; registered: {C}
*or still
t = n+2  stoppingstopped:{A, B}; starting, not registered: {C}

As you see, there will still be possible status that we have unregistered 
receiver C that processes data.

This pr implements another approach. The receivers register first then do 
starting process:

 tracker   receivers
t = 1  started   registered, started:{A, B}; registered, starting: 
{C}
t = 2  stoppinggot stop msg:{A, B, C}; **D wants to register -> 
timeout
t = 3  stoppingstopped:{A, B, C}




---
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: Added support for accessing secured HDFS

2015-02-11 Thread nchammas
Github user nchammas commented on the pull request:

https://github.com/apache/spark/pull/2320#issuecomment-74010939
  
This PR has gone stale. Do we want to update it or close it out?

Also, @huozhanfeng, could you update the title of the PR to "[SPARK-3438] 
Add support for accessing secured HDFS"? This matches our current convention.


---
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-5573][SQL] Add explode to dataframes

2015-02-11 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/4546#discussion_r2464
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala 
---
@@ -282,6 +282,33 @@ private[sql] class DataFrameImpl protected[sql](
 Sample(fraction, withReplacement, seed, logicalPlan)
   }
 
+  override def explode[A <: Product : TypeTag]
+  (input: Column*)(f: Row => TraversableOnce[A]): DataFrame = {
+val schema = 
ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType]
+val attributes = schema.toAttributes
+val rowFunction =
+  f.andThen(_.map(ScalaReflection.convertToCatalyst(_, 
schema).asInstanceOf[Row]))
+val generator = UserDefinedGenerator(attributes, rowFunction, 
input.map(_.expr))
+
+Generate(generator, join = true, outer = false, None, logicalPlan)
+  }
+
+  override def explode[A, B : TypeTag](
+  inputColumn: String,
+  outputColumn: String)(
+  f: A => TraversableOnce[B]): DataFrame = {
+val dataType = ScalaReflection.schemaFor[B].dataType
+val attributes = AttributeReference(outputColumn, dataType)() :: Nil
+def rowFunction(row: Row) = {
--- End diff --

if you define this as a val, i think it doesn't capture outer?



---
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-3299][SQL]Public API in SQLContext to l...

2015-02-11 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/4547#discussion_r2446
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
 ---
@@ -34,6 +34,12 @@ trait Catalog {
   tableIdentifier: Seq[String],
   alias: Option[String] = None): LogicalPlan
 
+  /**
+   * Returns names and flags indicating if a table is temporary or not of 
all tables in the
+   * database identified by `databaseIdentifier`.
+   */
+  def getTables(databaseIdentifier: Seq[String]): Seq[(String, Boolean)]
--- End diff --

I am following the convention of `tableIdentifier` at here (see 
https://issues.apache.org/jira/browse/SPARK-4943). In future we may want to 
support something like `catalog.database`.


---
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-5573][SQL] Add explode to dataframes

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4546#issuecomment-74010602
  
  [Test build #27318 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27318/consoleFull)
 for   PR 4546 at commit 
[`a8d496c`](https://github.com/apache/spark/commit/a8d496cb352c0b9423eeebeee82df69a5c97d2ea).
 * This patch merges cleanly.


---
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-5527] Improvements to standalone doc -e...

2015-02-11 Thread lazyman500
Github user lazyman500 commented on the pull request:

https://github.com/apache/spark/pull/4310#issuecomment-74010582
  
okey! 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-3570] Include time to open files in shu...

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4550#issuecomment-74010601
  
  [Test build #27317 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27317/consoleFull)
 for   PR 4550 at commit 
[`2423555`](https://github.com/apache/spark/commit/24235553adc748ccedf3998b0c174711b214e970).
 * This patch merges cleanly.


---
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-5527] Improvements to standalone doc -e...

2015-02-11 Thread lazyman500
Github user lazyman500 closed the pull request at:

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


---
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-3570] Include time to open files in shu...

2015-02-11 Thread kayousterhout
GitHub user kayousterhout opened a pull request:

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

[SPARK-3570] Include time to open files in shuffle write time.

Opening shuffle files can be very significant when the disk is
contended, especially when using ext3. While writing data to
a file can avoid hitting disk (and instead hit the buffer
cache), opening a file always involves writing some metadata
about the file to disk, so the open time can be a very significant
portion of the shuffle write time. In one job I ran recently, the time to
write shuffle data to the file was only 4ms for each task, but
the time to open the file was about 100x as long (~400ms).

When we add metrics about spilled data (#2504), we should ensure
that the file open time is also included there.

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

$ git pull https://github.com/kayousterhout/spark-1 SPARK-3570

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

https://github.com/apache/spark/pull/4550.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 #4550


commit 24235553adc748ccedf3998b0c174711b214e970
Author: Kay Ousterhout 
Date:   2014-09-17T22:07:11Z

[SPARK-3570] Include time to open files in shuffle write time.

Opening shuffle files can be very significant when the disk is
contended, especially when using ext3. While writing data to
a file can avoid hitting disk (and instead hit the buffer
cache), opening a file always involves writing some metadata
about the file to disk, so the open time can be a very significant
portion of the shuffle write time. In one recent job, the time to
write shuffle data to the file was only 4ms for each task, but
the time to open the file was about 100x as long (~400ms).

When we added metrics about spilled data (#2504), we should ensure
that the file open time is also included there.




---
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] Make dataframe more tolerant of being se...

2015-02-11 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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-5573][SQL] Add explode to dataframes

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4546#issuecomment-74009715
  
  [Test build #27315 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27315/consoleFull)
 for   PR 4546 at commit 
[`d633d01`](https://github.com/apache/spark/commit/d633d0162ac034e4d1b642980a152354ef452f57).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class UserDefinedGenerator(`
  * `   *   case class Book(title: String, words: String)`
  * `   *   case class Word(word: String)`



---
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-5573][SQL] Add explode to dataframes

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4546#issuecomment-74009718
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27315/
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-5722][SQL] fix for infer long type in p...

2015-02-11 Thread dondrake
Github user dondrake commented on the pull request:

https://github.com/apache/spark/pull/4521#issuecomment-74009322
  
OK, all int's will be inferred as LongType.  Ready for testing.


---
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-5573][SQL] Add explode to dataframes

2015-02-11 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/4546#issuecomment-74008987
  
  [Test build #27314 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/27314/consoleFull)
 for   PR 4546 at commit 
[`950707a`](https://github.com/apache/spark/commit/950707afbe59c976ed3ad044acfe31b7752eaa98).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `case class UserDefinedGenerator(`
  * `   *   case class Book(title: String, words: String)`
  * `   *   case class Word(word: String)`



---
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-5573][SQL] Add explode to dataframes

2015-02-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/4546#issuecomment-74008991
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/27314/
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



  1   2   3   4   5   >